diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 1767137b09..03db7c9fbc 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -214,6 +214,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.BitwiseNot.enabled` | Enable Comet acceleration for `BitwiseNot` | true | | `spark.comet.expression.BitwiseOr.enabled` | Enable Comet acceleration for `BitwiseOr` | true | | `spark.comet.expression.BitwiseXor.enabled` | Enable Comet acceleration for `BitwiseXor` | true | +| `spark.comet.expression.BloomFilterMightContain.enabled` | Enable Comet acceleration for `BloomFilterMightContain` | true | | `spark.comet.expression.CaseWhen.enabled` | Enable Comet acceleration for `CaseWhen` | true | | `spark.comet.expression.Cast.enabled` | Enable Comet acceleration for `Cast` | true | | `spark.comet.expression.Ceil.enabled` | Enable Comet acceleration for `Ceil` | true | @@ -258,6 +259,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.IsNaN.enabled` | Enable Comet acceleration for `IsNaN` | true | | `spark.comet.expression.IsNotNull.enabled` | Enable Comet acceleration for `IsNotNull` | true | | `spark.comet.expression.IsNull.enabled` | Enable Comet acceleration for `IsNull` | true | +| `spark.comet.expression.KnownFloatingPointNormalized.enabled` | Enable Comet acceleration for `KnownFloatingPointNormalized` | true | | `spark.comet.expression.Length.enabled` | Enable Comet acceleration for `Length` | true | | `spark.comet.expression.LessThan.enabled` | Enable Comet acceleration for `LessThan` | true | | `spark.comet.expression.LessThanOrEqual.enabled` | Enable Comet acceleration for `LessThanOrEqual` | true | @@ -267,6 +269,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.Log10.enabled` | Enable Comet acceleration for `Log10` | true | | `spark.comet.expression.Log2.enabled` | Enable Comet acceleration for `Log2` | true | | `spark.comet.expression.Lower.enabled` | Enable Comet acceleration for `Lower` | true | +| `spark.comet.expression.MakeDecimal.enabled` | Enable Comet acceleration for `MakeDecimal` | true | | `spark.comet.expression.MapEntries.enabled` | Enable Comet acceleration for `MapEntries` | true | | `spark.comet.expression.MapFromArrays.enabled` | Enable Comet acceleration for `MapFromArrays` | true | | `spark.comet.expression.MapKeys.enabled` | Enable Comet acceleration for `MapKeys` | true | @@ -289,6 +292,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.Remainder.enabled` | Enable Comet acceleration for `Remainder` | true | | `spark.comet.expression.Reverse.enabled` | Enable Comet acceleration for `Reverse` | true | | `spark.comet.expression.Round.enabled` | Enable Comet acceleration for `Round` | true | +| `spark.comet.expression.ScalarSubquery.enabled` | Enable Comet acceleration for `ScalarSubquery` | true | | `spark.comet.expression.Second.enabled` | Enable Comet acceleration for `Second` | true | | `spark.comet.expression.Sha1.enabled` | Enable Comet acceleration for `Sha1` | true | | `spark.comet.expression.Sha2.enabled` | Enable Comet acceleration for `Sha2` | true | @@ -320,6 +324,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.TruncTimestamp.enabled` | Enable Comet acceleration for `TruncTimestamp` | true | | `spark.comet.expression.UnaryMinus.enabled` | Enable Comet acceleration for `UnaryMinus` | true | | `spark.comet.expression.Unhex.enabled` | Enable Comet acceleration for `Unhex` | true | +| `spark.comet.expression.UnscaledValue.enabled` | Enable Comet acceleration for `UnscaledValue` | true | | `spark.comet.expression.Upper.enabled` | Enable Comet acceleration for `Upper` | true | | `spark.comet.expression.WeekDay.enabled` | Enable Comet acceleration for `WeekDay` | true | | `spark.comet.expression.WeekOfYear.enabled` | Enable Comet acceleration for `WeekOfYear` | true | diff --git a/spark/src/main/scala/org/apache/comet/serde/CometBloomFilterMightContain.scala b/spark/src/main/scala/org/apache/comet/serde/CometBloomFilterMightContain.scala new file mode 100644 index 0000000000..a13b685ea6 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/CometBloomFilterMightContain.scala @@ -0,0 +1,52 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Attribute, BloomFilterMightContain} + +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.QueryPlanSerde.exprToProtoInternal + +object CometBloomFilterMightContain extends CometExpressionSerde[BloomFilterMightContain] { + + override def convert( + expr: BloomFilterMightContain, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + + val bloomFilter = expr.left + val value = expr.right + val bloomFilterExpr = exprToProtoInternal(bloomFilter, inputs, binding) + val valueExpr = exprToProtoInternal(value, inputs, binding) + if (bloomFilterExpr.isDefined && valueExpr.isDefined) { + val builder = ExprOuterClass.BloomFilterMightContain.newBuilder() + builder.setBloomFilter(bloomFilterExpr.get) + builder.setValue(valueExpr.get) + Some( + ExprOuterClass.Expr + .newBuilder() + .setBloomFilterMightContain(builder) + .build()) + } else { + withInfo(expr, bloomFilter, value) + None + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/CometScalarSubquery.scala b/spark/src/main/scala/org/apache/comet/serde/CometScalarSubquery.scala new file mode 100644 index 0000000000..b1f5a28271 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/CometScalarSubquery.scala @@ -0,0 +1,51 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.ScalarSubquery + +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.QueryPlanSerde.{serializeDataType, supportedDataType} + +object CometScalarSubquery extends CometExpressionSerde[ScalarSubquery] { + override def convert( + expr: ScalarSubquery, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + if (supportedDataType(expr.dataType)) { + val dataType = serializeDataType(expr.dataType) + if (dataType.isEmpty) { + withInfo(expr, s"Failed to serialize datatype ${expr.dataType} for scalar subquery") + return None + } + + val builder = ExprOuterClass.Subquery + .newBuilder() + .setId(expr.exprId.id) + .setDatatype(dataType.get) + Some(ExprOuterClass.Expr.newBuilder().setSubquery(builder).build()) + } else { + withInfo(expr, s"Unsupported data type: ${expr.dataType}") + None + } + + } +} 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 6bf3776a23..0bcfb42d09 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -25,10 +25,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke -import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero -import org.apache.spark.sql.comet._ -import org.apache.spark.sql.execution -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.comet.DecimalPrecision +import org.apache.spark.sql.execution.{ScalarSubquery, SparkPlan} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -153,7 +151,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[Lower] -> CometLower, classOf[OctetLength] -> CometScalarFunction("octet_length"), classOf[RegExpReplace] -> CometRegExpReplace, - classOf[Reverse] -> CometScalarFunction("reverse"), + classOf[Reverse] -> CometReverse, classOf[RLike] -> CometRLike, classOf[StartsWith] -> CometScalarFunction("starts_with"), classOf[StringInstr] -> CometScalarFunction("instr"), @@ -203,21 +201,20 @@ object QueryPlanSerde extends Logging with CometExprShim { private val miscExpressions: Map[Class[_ <: Expression], CometExpressionSerde[_]] = Map( // TODO PromotePrecision - // TODO KnownFloatingPointNormalized - // TODO ScalarSubquery - // TODO UnscaledValue - // TODO MakeDecimal - // TODO BloomFilterMightContain - // TODO RegExpReplace classOf[Alias] -> CometAlias, classOf[AttributeReference] -> CometAttributeReference, + classOf[BloomFilterMightContain] -> CometBloomFilterMightContain, classOf[CheckOverflow] -> CometCheckOverflow, classOf[Coalesce] -> CometCoalesce, + classOf[KnownFloatingPointNormalized] -> CometKnownFloatingPointNormalized, classOf[Literal] -> CometLiteral, + classOf[MakeDecimal] -> CometMakeDecimal, classOf[MonotonicallyIncreasingID] -> CometMonotonicallyIncreasingId, + classOf[ScalarSubquery] -> CometScalarSubquery, classOf[SparkPartitionID] -> CometSparkPartitionId, classOf[SortOrder] -> CometSortOrder, - classOf[StaticInvoke] -> CometStaticInvoke) + classOf[StaticInvoke] -> CometStaticInvoke, + classOf[UnscaledValue] -> CometUnscaledValue) /** * Mapping of Spark expression class to Comet expression handler. @@ -541,74 +538,6 @@ object QueryPlanSerde extends Logging with CometExprShim { // `PromotePrecision` is just a wrapper, don't need to serialize it. exprToProtoInternal(child, inputs, binding) - case KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) => - val dataType = serializeDataType(expr.dataType) - if (dataType.isEmpty) { - withInfo(expr, s"Unsupported datatype ${expr.dataType}") - return None - } - val ex = exprToProtoInternal(expr, inputs, binding) - ex.map { child => - val builder = ExprOuterClass.NormalizeNaNAndZero - .newBuilder() - .setChild(child) - .setDatatype(dataType.get) - ExprOuterClass.Expr.newBuilder().setNormalizeNanAndZero(builder).build() - } - - case s @ execution.ScalarSubquery(_, _) => - if (supportedDataType(s.dataType)) { - val dataType = serializeDataType(s.dataType) - if (dataType.isEmpty) { - withInfo(s, s"Scalar subquery returns unsupported datatype ${s.dataType}") - return None - } - - val builder = ExprOuterClass.Subquery - .newBuilder() - .setId(s.exprId.id) - .setDatatype(dataType.get) - Some(ExprOuterClass.Expr.newBuilder().setSubquery(builder).build()) - } else { - withInfo(s, s"Unsupported data type: ${s.dataType}") - None - } - - case UnscaledValue(child) => - val childExpr = exprToProtoInternal(child, inputs, binding) - val optExpr = - scalarFunctionExprToProtoWithReturnType("unscaled_value", LongType, false, childExpr) - optExprWithInfo(optExpr, expr, child) - - case MakeDecimal(child, precision, scale, true) => - val childExpr = exprToProtoInternal(child, inputs, binding) - val optExpr = scalarFunctionExprToProtoWithReturnType( - "make_decimal", - DecimalType(precision, scale), - false, - childExpr) - optExprWithInfo(optExpr, expr, child) - - case b @ BloomFilterMightContain(_, _) => - val bloomFilter = b.left - val value = b.right - val bloomFilterExpr = exprToProtoInternal(bloomFilter, inputs, binding) - val valueExpr = exprToProtoInternal(value, inputs, binding) - if (bloomFilterExpr.isDefined && valueExpr.isDefined) { - val builder = ExprOuterClass.BloomFilterMightContain.newBuilder() - builder.setBloomFilter(bloomFilterExpr.get) - builder.setValue(valueExpr.get) - Some( - ExprOuterClass.Expr - .newBuilder() - .setBloomFilterMightContain(builder) - .build()) - } else { - withInfo(expr, bloomFilter, value) - None - } - case r @ Reverse(child) if child.dataType.isInstanceOf[ArrayType] => - convert(r, CometArrayReverse) case expr => QueryPlanSerde.exprSerdeMap.get(expr.getClass) match { case Some(handler) => diff --git a/spark/src/main/scala/org/apache/comet/serde/collectionOperations.scala b/spark/src/main/scala/org/apache/comet/serde/collectionOperations.scala new file mode 100644 index 0000000000..716241040e --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/collectionOperations.scala @@ -0,0 +1,44 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Reverse} +import org.apache.spark.sql.types.ArrayType + +import org.apache.comet.serde.ExprOuterClass.Expr + +object CometReverse extends CometScalarFunction[Reverse]("reverse") { + + override def getSupportLevel(expr: Reverse): SupportLevel = { + if (expr.child.dataType.isInstanceOf[ArrayType]) { + CometArrayReverse.getSupportLevel(expr) + } else { + Compatible() + } + } + + override def convert(expr: Reverse, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { + if (expr.child.dataType.isInstanceOf[ArrayType]) { + CometArrayReverse.convert(expr, inputs, binding) + } else { + super.convert(expr, inputs, binding) + } + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/contraintExpressions.scala b/spark/src/main/scala/org/apache/comet/serde/contraintExpressions.scala new file mode 100644 index 0000000000..62643d6da6 --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/contraintExpressions.scala @@ -0,0 +1,63 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Attribute, KnownFloatingPointNormalized} +import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero + +import org.apache.comet.CometSparkSessionExtensions.withInfo +import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, serializeDataType} + +object CometKnownFloatingPointNormalized + extends CometExpressionSerde[KnownFloatingPointNormalized] { + + override def getSupportLevel(expr: KnownFloatingPointNormalized): SupportLevel = { + expr.child match { + case _: NormalizeNaNAndZero => Compatible() + case _ => + Unsupported( + Some( + "KnownFloatingPointNormalized only supports NormalizeNaNAndZero child expressions")) + } + } + + override def convert( + expr: KnownFloatingPointNormalized, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + + val wrapped = expr.child.asInstanceOf[NormalizeNaNAndZero].child + + val dataType = serializeDataType(wrapped.dataType) + if (dataType.isEmpty) { + withInfo(wrapped, s"Unsupported datatype ${wrapped.dataType}") + return None + } + val ex = exprToProtoInternal(wrapped, inputs, binding) + val optExpr = ex.map { child => + val builder = ExprOuterClass.NormalizeNaNAndZero + .newBuilder() + .setChild(child) + .setDatatype(dataType.get) + ExprOuterClass.Expr.newBuilder().setNormalizeNanAndZero(builder).build() + } + optExprWithInfo(optExpr, expr, wrapped) + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/decimalExpressions.scala b/spark/src/main/scala/org/apache/comet/serde/decimalExpressions.scala new file mode 100644 index 0000000000..c606d1ac5b --- /dev/null +++ b/spark/src/main/scala/org/apache/comet/serde/decimalExpressions.scala @@ -0,0 +1,54 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.{Attribute, MakeDecimal, UnscaledValue} +import org.apache.spark.sql.types.{DecimalType, LongType} + +import org.apache.comet.serde.QueryPlanSerde.{exprToProtoInternal, optExprWithInfo, scalarFunctionExprToProtoWithReturnType} + +object CometUnscaledValue extends CometExpressionSerde[UnscaledValue] { + override def convert( + expr: UnscaledValue, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val childExpr = exprToProtoInternal(expr.child, inputs, binding) + val optExpr = + scalarFunctionExprToProtoWithReturnType("unscaled_value", LongType, false, childExpr) + optExprWithInfo(optExpr, expr, expr.child) + + } +} + +object CometMakeDecimal extends CometExpressionSerde[MakeDecimal] { + override def convert( + expr: MakeDecimal, + inputs: Seq[Attribute], + binding: Boolean): Option[ExprOuterClass.Expr] = { + val childExpr = exprToProtoInternal(expr.child, inputs, binding) + val optExpr = scalarFunctionExprToProtoWithReturnType( + "make_decimal", + DecimalType(expr.precision, expr.scale), + failOnError = !expr.nullOnOverflow, + childExpr) + optExprWithInfo(optExpr, expr, expr.child) + + } +} diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt index 59af19b16a..a37054da2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/explain.txt @@ -1,49 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (28) - : : +- * Filter (27) - : : +- * HashAggregate (26) - : : +- * CometColumnarToRow (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (14) - : : +- ReusedExchange (16) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- BroadcastExchange (42) - +- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) +* CometColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometFilter (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (25) + : : +- CometFilter (24) + : : +- CometHashAggregate (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometHashAggregate (20) + : : +- CometExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (15) + : +- CometBroadcastExchange (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns @@ -95,202 +91,179 @@ Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(12) HashAggregate [codegen id : 5] +(11) CometHashAggregate Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(13) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) +(12) CometFilter +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Condition : isnotnull(ctr_total_return#11) -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Condition : isnotnull(sr_store_sk#14) - -(16) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#18] - -(17) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [sr_returned_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(18) CometProject -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18] -Arguments: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15], [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] - -(19) CometHashAggregate -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] - -(20) CometExchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(22) HashAggregate [codegen id : 1] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] -Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] - -(23) HashAggregate [codegen id : 1] -Input [2]: [ctr_store_sk#20, ctr_total_return#21] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [partial_avg(ctr_total_return#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_store_sk#20, sum#24, count#25] - -(24) CometColumnarExchange -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(25) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_store_sk#20, sum#24, count#25] - -(26) HashAggregate [codegen id : 2] -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [avg(ctr_total_return#21)] -Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] -Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(27) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#20] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) - -(30) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#28, s_state#29] +(14) CometFilter +Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Condition : isnotnull(sr_store_sk#13) + +(15) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#17] + +(16) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight + +(17) CometProject +Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] +Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] + +(18) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] + +(19) CometExchange +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(20) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] + +(21) CometHashAggregate +Input [2]: [ctr_store_sk#19, ctr_total_return#20] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [partial_avg(ctr_total_return#20)] + +(22) CometExchange +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(23) CometHashAggregate +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [avg(ctr_total_return#20)] + +(24) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) + +(25) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] + +(26) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight + +(27) CometProject +Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] + +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [s_store_sk#28, s_state#29] -Condition : ((isnotnull(s_state#29) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)) = TN)) AND isnotnull(s_store_sk#28)) +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : ((isnotnull(s_state#25) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) = TN)) AND isnotnull(s_store_sk#24)) -(33) CometProject -Input [2]: [s_store_sk#28, s_state#29] -Arguments: [s_store_sk#28], [s_store_sk#28] +(30) CometProject +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24], [s_store_sk#24] -(34) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#28] +(31) CometBroadcastExchange +Input [1]: [s_store_sk#24] +Arguments: [s_store_sk#24] -(35) BroadcastExchange -Input [1]: [s_store_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(32) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] +Right output [1]: [s_store_sk#24] +Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight -(36) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 5] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] +(33) CometProject +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] +Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#30, c_customer_id#31] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#26, c_customer_id#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [c_customer_sk#30, c_customer_id#31] -Condition : isnotnull(c_customer_sk#30) +(35) CometFilter +Input [2]: [c_customer_sk#26, c_customer_id#27] +Condition : isnotnull(c_customer_sk#26) -(40) CometProject -Input [2]: [c_customer_sk#30, c_customer_id#31] -Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#31, 16)) AS c_customer_id#32] +(36) CometProject +Input [2]: [c_customer_sk#26, c_customer_id#27] +Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#28] -(41) CometColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#30, c_customer_id#32] +(37) CometBroadcastExchange +Input [2]: [c_customer_sk#26, c_customer_id#28] +Arguments: [c_customer_sk#26, c_customer_id#28] -(42) BroadcastExchange -Input [2]: [c_customer_sk#30, c_customer_id#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(38) CometBroadcastHashJoin +Left output [1]: [ctr_customer_sk#9] +Right output [2]: [c_customer_sk#26, c_customer_id#28] +Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#30] -Join type: Inner -Join condition: None +(39) CometProject +Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] +Arguments: [c_customer_id#28], [c_customer_id#28] -(44) Project [codegen id : 5] -Output [1]: [c_customer_id#32] -Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] +(40) CometTakeOrderedAndProject +Input [1]: [c_customer_id#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] -(45) TakeOrderedAndProject -Input [1]: [c_customer_id#32] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] +(41) CometColumnarToRow [codegen id : 1] +Input [1]: [c_customer_id#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(47) CometFilter +(43) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(48) CometProject +(44) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(49) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(50) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt index d483ace466..8aa14c43dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/extended.txt @@ -1,57 +1,53 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(sr_customer_sk#1 AS ctr_customer_sk#2, sr_store_sk#3 AS ctr_store_sk#4, MakeDecimal(sum(UnscaledValue(sr_return_amt#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- 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 - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(sr_store_sk#8 AS ctr_store_sk#9, MakeDecimal(sum(UnscaledValue(sr_return_amt#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 30 out of 49 eligible operators (61%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt index 0d694f3f2e..3e1d6243f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1.native_iceberg_compat/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (5) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id] + CometProject [c_customer_id] + CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] + CometProject [ctr_customer_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] + CometProject [ctr_customer_sk,ctr_store_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #1 CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] @@ -28,40 +28,25 @@ TakeOrderedAndProject [c_customer_id] 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 #4 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #5 - WholeStageCodegen (1) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #7 - 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] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] + CometExchange [ctr_store_sk] #5 + CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] + CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] + CometExchange [sr_customer_sk,sr_store_sk] #6 + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + 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] + CometBroadcastExchange [c_customer_sk,c_customer_id] #8 + CometProject [c_customer_id] [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk,c_customer_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt index 59af19b16a..a37054da2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt @@ -1,49 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (45) -+- * Project (44) - +- * BroadcastHashJoin Inner BuildRight (43) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Project (30) - : : +- * BroadcastHashJoin Inner BuildRight (29) - : : :- * Filter (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (28) - : : +- * Filter (27) - : : +- * HashAggregate (26) - : : +- * CometColumnarToRow (25) - : : +- CometColumnarExchange (24) - : : +- * HashAggregate (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (15) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (14) - : : +- ReusedExchange (16) - : +- BroadcastExchange (35) - : +- * CometColumnarToRow (34) - : +- CometProject (33) - : +- CometFilter (32) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (31) - +- BroadcastExchange (42) - +- * CometColumnarToRow (41) - +- CometProject (40) - +- CometFilter (39) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) +* CometColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometProject (39) + +- CometBroadcastHashJoin (38) + :- CometProject (33) + : +- CometBroadcastHashJoin (32) + : :- CometProject (27) + : : +- CometBroadcastHashJoin (26) + : : :- CometFilter (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (25) + : : +- CometFilter (24) + : : +- CometHashAggregate (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometHashAggregate (20) + : : +- CometExchange (19) + : : +- CometHashAggregate (18) + : : +- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (13) + : : +- ReusedExchange (15) + : +- CometBroadcastExchange (31) + : +- CometProject (30) + : +- CometFilter (29) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (28) + +- CometBroadcastExchange (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (34) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns @@ -95,202 +91,179 @@ Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#3))] Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometColumnarToRow [codegen id : 5] -Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] - -(12) HashAggregate [codegen id : 5] +(11) CometHashAggregate Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] Keys [2]: [sr_customer_sk#1, sr_store_sk#2] Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] -Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] -(13) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] -Condition : isnotnull(ctr_total_return#12) +(12) CometFilter +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Condition : isnotnull(ctr_total_return#11) -(14) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#17)] +PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#16)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(15) CometFilter -Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Condition : isnotnull(sr_store_sk#14) - -(16) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#18] - -(17) CometBroadcastHashJoin -Left output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] -Right output [1]: [d_date_sk#18] -Arguments: [sr_returned_date_sk#16], [d_date_sk#18], Inner, BuildRight - -(18) CometProject -Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18] -Arguments: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15], [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] - -(19) CometHashAggregate -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] - -(20) CometExchange -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(21) CometColumnarToRow [codegen id : 1] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] - -(22) HashAggregate [codegen id : 1] -Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#19] -Keys [2]: [sr_customer_sk#13, sr_store_sk#14] -Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] -Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] -Results [2]: [sr_store_sk#14 AS ctr_store_sk#20, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#21] - -(23) HashAggregate [codegen id : 1] -Input [2]: [ctr_store_sk#20, ctr_total_return#21] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [partial_avg(ctr_total_return#21)] -Aggregate Attributes [2]: [sum#22, count#23] -Results [3]: [ctr_store_sk#20, sum#24, count#25] - -(24) CometColumnarExchange -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Arguments: hashpartitioning(ctr_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(25) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_store_sk#20, sum#24, count#25] - -(26) HashAggregate [codegen id : 2] -Input [3]: [ctr_store_sk#20, sum#24, count#25] -Keys [1]: [ctr_store_sk#20] -Functions [1]: [avg(ctr_total_return#21)] -Aggregate Attributes [1]: [avg(ctr_total_return#21)#26] -Results [2]: [(avg(ctr_total_return#21)#26 * 1.2) AS (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(27) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#27) - -(28) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] - -(29) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [ctr_store_sk#20] -Join type: Inner -Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#27) - -(30) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] -Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#27, ctr_store_sk#20] - -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#28, s_state#29] +(14) CometFilter +Input [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Condition : isnotnull(sr_store_sk#13) + +(15) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#17] + +(16) CometBroadcastHashJoin +Left output [4]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15] +Right output [1]: [d_date_sk#17] +Arguments: [sr_returned_date_sk#15], [d_date_sk#17], Inner, BuildRight + +(17) CometProject +Input [5]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14, sr_returned_date_sk#15, d_date_sk#17] +Arguments: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14], [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] + +(18) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sr_return_amt#14] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#14))] + +(19) CometExchange +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Arguments: hashpartitioning(sr_customer_sk#12, sr_store_sk#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(20) CometHashAggregate +Input [3]: [sr_customer_sk#12, sr_store_sk#13, sum#18] +Keys [2]: [sr_customer_sk#12, sr_store_sk#13] +Functions [1]: [sum(UnscaledValue(sr_return_amt#14))] + +(21) CometHashAggregate +Input [2]: [ctr_store_sk#19, ctr_total_return#20] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [partial_avg(ctr_total_return#20)] + +(22) CometExchange +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Arguments: hashpartitioning(ctr_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(23) CometHashAggregate +Input [3]: [ctr_store_sk#19, sum#21, count#22] +Keys [1]: [ctr_store_sk#19] +Functions [1]: [avg(ctr_total_return#20)] + +(24) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#23) + +(25) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] + +(26) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11] +Right output [2]: [(avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_store_sk#10], [ctr_store_sk#19], Inner, (cast(ctr_total_return#11 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#23), BuildRight + +(27) CometProject +Input [5]: [ctr_customer_sk#9, ctr_store_sk#10, ctr_total_return#11, (avg(ctr_total_return) * 1.2)#23, ctr_store_sk#19] +Arguments: [ctr_customer_sk#9, ctr_store_sk#10], [ctr_customer_sk#9, ctr_store_sk#10] + +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#24, s_state#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_state), IsNotNull(s_store_sk)] ReadSchema: struct -(32) CometFilter -Input [2]: [s_store_sk#28, s_state#29] -Condition : ((isnotnull(s_state#29) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#29, 2)) = TN)) AND isnotnull(s_store_sk#28)) +(29) CometFilter +Input [2]: [s_store_sk#24, s_state#25] +Condition : ((isnotnull(s_state#25) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#25, 2)) = TN)) AND isnotnull(s_store_sk#24)) -(33) CometProject -Input [2]: [s_store_sk#28, s_state#29] -Arguments: [s_store_sk#28], [s_store_sk#28] +(30) CometProject +Input [2]: [s_store_sk#24, s_state#25] +Arguments: [s_store_sk#24], [s_store_sk#24] -(34) CometColumnarToRow [codegen id : 3] -Input [1]: [s_store_sk#28] +(31) CometBroadcastExchange +Input [1]: [s_store_sk#24] +Arguments: [s_store_sk#24] -(35) BroadcastExchange -Input [1]: [s_store_sk#28] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(32) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#9, ctr_store_sk#10] +Right output [1]: [s_store_sk#24] +Arguments: [ctr_store_sk#10], [s_store_sk#24], Inner, BuildRight -(36) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_store_sk#11] -Right keys [1]: [s_store_sk#28] -Join type: Inner -Join condition: None - -(37) Project [codegen id : 5] -Output [1]: [ctr_customer_sk#10] -Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#28] +(33) CometProject +Input [3]: [ctr_customer_sk#9, ctr_store_sk#10, s_store_sk#24] +Arguments: [ctr_customer_sk#9], [ctr_customer_sk#9] -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [2]: [c_customer_sk#30, c_customer_id#31] +(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#26, c_customer_id#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [2]: [c_customer_sk#30, c_customer_id#31] -Condition : isnotnull(c_customer_sk#30) +(35) CometFilter +Input [2]: [c_customer_sk#26, c_customer_id#27] +Condition : isnotnull(c_customer_sk#26) -(40) CometProject -Input [2]: [c_customer_sk#30, c_customer_id#31] -Arguments: [c_customer_sk#30, c_customer_id#32], [c_customer_sk#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#31, 16)) AS c_customer_id#32] +(36) CometProject +Input [2]: [c_customer_sk#26, c_customer_id#27] +Arguments: [c_customer_sk#26, c_customer_id#28], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#28] -(41) CometColumnarToRow [codegen id : 4] -Input [2]: [c_customer_sk#30, c_customer_id#32] +(37) CometBroadcastExchange +Input [2]: [c_customer_sk#26, c_customer_id#28] +Arguments: [c_customer_sk#26, c_customer_id#28] -(42) BroadcastExchange -Input [2]: [c_customer_sk#30, c_customer_id#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(38) CometBroadcastHashJoin +Left output [1]: [ctr_customer_sk#9] +Right output [2]: [c_customer_sk#26, c_customer_id#28] +Arguments: [ctr_customer_sk#9], [c_customer_sk#26], Inner, BuildRight -(43) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#10] -Right keys [1]: [c_customer_sk#30] -Join type: Inner -Join condition: None +(39) CometProject +Input [3]: [ctr_customer_sk#9, c_customer_sk#26, c_customer_id#28] +Arguments: [c_customer_id#28], [c_customer_id#28] -(44) Project [codegen id : 5] -Output [1]: [c_customer_id#32] -Input [3]: [ctr_customer_sk#10, c_customer_sk#30, c_customer_id#32] +(40) CometTakeOrderedAndProject +Input [1]: [c_customer_id#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#28 ASC NULLS FIRST], output=[c_customer_id#28]), [c_customer_id#28], 100, 0, [c_customer_id#28 ASC NULLS FIRST], [c_customer_id#28] -(45) TakeOrderedAndProject -Input [1]: [c_customer_id#32] -Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] +(41) CometColumnarToRow [codegen id : 1] +Input [1]: [c_customer_id#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (50) -+- * CometColumnarToRow (49) - +- CometProject (48) - +- CometFilter (47) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (46) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(47) CometFilter +(43) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(48) CometProject +(44) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(49) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(50) BroadcastExchange +(46) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 14 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 13 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt index d483ace466..8aa14c43dd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/extended.txt @@ -1,57 +1,53 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(sr_customer_sk#1 AS ctr_customer_sk#2, sr_store_sk#3 AS ctr_store_sk#4, MakeDecimal(sum(UnscaledValue(sr_return_amt#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : : +- 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 - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(sr_store_sk#8 AS ctr_store_sk#9, MakeDecimal(sum(UnscaledValue(sr_return_amt#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 30 out of 49 eligible operators (61%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 49 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt index 0d694f3f2e..3e1d6243f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id] - WholeStageCodegen (5) - Project [c_customer_id] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk] - BroadcastHashJoin [ctr_store_sk,s_store_sk] - Project [ctr_customer_sk,ctr_store_sk] - BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id] + CometProject [c_customer_id] + CometBroadcastHashJoin [ctr_customer_sk,c_customer_sk,c_customer_id] + CometProject [ctr_customer_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,s_store_sk] + CometProject [ctr_customer_sk,ctr_store_sk] + CometBroadcastHashJoin [ctr_customer_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometFilter [ctr_customer_sk,ctr_store_sk,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_store_sk,ctr_total_return,sr_customer_sk,sr_store_sk,sum(UnscaledValue(sr_return_amt))] CometExchange [sr_customer_sk,sr_store_sk] #1 CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] @@ -28,40 +28,25 @@ TakeOrderedAndProject [c_customer_id] 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 #4 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_store_sk] #5 - WholeStageCodegen (1) - HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] - HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [sr_customer_sk,sr_store_sk] #6 - CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] - CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] - CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] - CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - InputAdapter - BroadcastExchange #7 - 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] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id] [c_customer_sk,c_customer_id] - CometFilter [c_customer_sk,c_customer_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_store_sk] #4 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_store_sk] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_store_sk,avg(ctr_total_return)] + CometExchange [ctr_store_sk] #5 + CometHashAggregate [ctr_total_return] [ctr_store_sk,sum,count] + CometHashAggregate [sr_customer_sk,sum] [ctr_store_sk,ctr_total_return,sr_store_sk,sum(UnscaledValue(sr_return_amt))] + CometExchange [sr_customer_sk,sr_store_sk] #6 + CometHashAggregate [sr_return_amt] [sr_customer_sk,sr_store_sk,sum] + CometProject [sr_customer_sk,sr_store_sk,sr_return_amt] + CometBroadcastHashJoin [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk,d_date_sk] + CometFilter [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + 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] + CometBroadcastExchange [c_customer_sk,c_customer_id] #8 + CometProject [c_customer_id] [c_customer_sk,c_customer_id] + CometFilter [c_customer_sk,c_customer_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt index b9d337c55b..b6edfa8800 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/explain.txt @@ -1,80 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Project (75) - +- * BroadcastHashJoin Inner BuildRight (74) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (55) - : +- * Filter (54) - : +- * HashAggregate (53) - : +- * CometColumnarToRow (52) - : +- CometExchange (51) - : +- CometHashAggregate (50) - : +- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (39) - : : +- CometBroadcastExchange (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (47) - +- BroadcastExchange (73) - +- * HashAggregate (72) - +- * CometColumnarToRow (71) - +- CometExchange (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometBroadcastHashJoin (67) - :- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (60) - : : +- CometFilter (59) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (58) - : +- CometBroadcastExchange (63) - : +- CometFilter (62) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (61) - +- ReusedExchange (66) +* CometColumnarToRow (73) ++- CometTakeOrderedAndProject (72) + +- CometProject (71) + +- CometBroadcastHashJoin (70) + :- CometProject (54) + : +- CometBroadcastHashJoin (53) + : :- CometProject (36) + : : +- 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 (52) + : +- CometFilter (51) + : +- CometHashAggregate (50) + : +- CometExchange (49) + : +- CometHashAggregate (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometProject (39) + : : : +- CometFilter (38) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + : : +- CometBroadcastExchange (42) + : : +- CometFilter (41) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) + : +- ReusedExchange (45) + +- CometBroadcastExchange (69) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) + : +- CometBroadcastExchange (60) + : +- CometFilter (59) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) + +- ReusedExchange (63) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -150,349 +147,329 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discoun 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 : 4] -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] - -(17) HashAggregate [codegen id : 4] +(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)))#23] -Results [2]: [c_customer_id#9 AS customer_id#24, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#23,18,2) AS year_total#25] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#24, year_total#25] -Condition : (isnotnull(year_total#25) AND (year_total#25 > 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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] +(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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) -(21) CometProject -Input [8]: [c_customer_sk#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Arguments: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#30, 1)) AS c_preferred_cust_flag#37, c_birth_country#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#32, 13)) AS c_login#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#33, 50)) AS c_email_address#39] +(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#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] 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#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Condition : isnotnull(ss_customer_sk#40) +(22) CometFilter +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Condition : isnotnull(ss_customer_sk#39) -(24) CometBroadcastExchange -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(25) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Right output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_sk#26], [ss_customer_sk#40], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight -(26) CometProject -Input [12]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(26) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: [d_date_sk#45, d_year#46] - -(30) CometBroadcastHashJoin -Left output [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Right output [2]: [d_date_sk#45, d_year#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#45], Inner, BuildRight - -(31) CometProject -Input [12]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43, d_date_sk#45, d_year#46] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] - -(32) CometHashAggregate -Input [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] - -(33) CometExchange -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] - -(35) HashAggregate [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23] -Results [3]: [c_customer_id#34 AS customer_id#48, c_preferred_cust_flag#37 AS customer_preferred_cust_flag#49, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23,18,2) AS year_total#50] - -(36) BroadcastExchange -Input [3]: [customer_id#48, customer_preferred_cust_flag#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#48] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [4]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50] -Input [5]: [customer_id#24, year_total#25, customer_id#48, customer_preferred_cust_flag#49, year_total#50] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +(27) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] + +(31) CometHashAggregate +Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(32) CometExchange +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(34) CometBroadcastExchange +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight + +(36) CometProject +Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) +(38) CometFilter +Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Condition : (isnotnull(c_customer_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)))) -(41) CometProject -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Arguments: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#55, 1)) AS c_preferred_cust_flag#62, c_birth_country#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#57, 13)) AS c_login#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#58, 50)) AS c_email_address#64] +(39) CometProject +Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Arguments: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63], [c_customer_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)) AS c_customer_id#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#52, 20)) AS c_first_name#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#53, 30)) AS c_last_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#54, 1)) AS c_preferred_cust_flag#61, c_birth_country#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#56, 13)) AS c_login#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#57, 50)) AS c_email_address#63] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] 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#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_bill_customer_sk#65) +(41) CometFilter +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_customer_sk#64) -(44) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(42) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(45) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64] -Right output [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#65], Inner, BuildRight +(43) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63] +Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_sk#50], [ws_bill_customer_sk#64], Inner, BuildRight -(46) CometProject -Input [12]: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(44) CometProject +Input [12]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(47) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#70, d_year#71] +(45) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#69, d_year#70] -(48) CometBroadcastHashJoin -Left output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Right output [2]: [d_date_sk#70, d_year#71] -Arguments: [ws_sold_date_sk#68], [d_date_sk#70], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Right output [2]: [d_date_sk#69, d_year#70] +Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight -(49) CometProject -Input [12]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68, d_date_sk#70, d_year#71] -Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71] +(47) CometProject +Input [12]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] +Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] + +(48) CometHashAggregate +Input [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(49) CometExchange +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] +Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (50) CometHashAggregate -Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))] - -(51) CometExchange -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometColumnarToRow [codegen id : 2] -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] - -(53) HashAggregate [codegen id : 2] -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))#73] -Results [2]: [c_customer_id#59 AS customer_id#74, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))#73,18,2) AS year_total#75] - -(54) Filter [codegen id : 2] -Input [2]: [customer_id#74, year_total#75] -Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) - -(55) BroadcastExchange -Input [2]: [customer_id#74, year_total#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(56) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#74] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 4] -Output [5]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, year_total#75] -Input [6]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, customer_id#74, year_total#75] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(51) CometFilter +Input [2]: [customer_id#72, year_total#73] +Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) + +(52) CometBroadcastExchange +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72, year_total#73] + +(53) CometBroadcastHashJoin +Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] +Right output [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#23], [customer_id#72], Inner, BuildRight + +(54) CometProject +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] +Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] + +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(59) CometFilter -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) +(56) CometFilter +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Condition : (isnotnull(c_customer_sk#74) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)))) -(60) CometProject -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] +(57) CometProject +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Arguments: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87], [c_customer_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)) AS c_customer_id#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#76, 20)) AS c_first_name#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#77, 30)) AS c_last_name#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#78, 1)) AS c_preferred_cust_flag#85, c_birth_country#79, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#80, 13)) AS c_login#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#81, 50)) AS c_email_address#87] -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(62) CometFilter -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_bill_customer_sk#90) +(59) CometFilter +Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_bill_customer_sk#88) + +(60) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] + +(61) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87] +Right output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [c_customer_sk#74], [ws_bill_customer_sk#88], Inner, BuildRight + +(62) CometProject +Input [12]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -(63) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +(63) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#93, d_year#94] (64) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] -Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight +Left output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Right output [2]: [d_date_sk#93, d_year#94] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight (65) CometProject -Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(66) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#95, d_year#96] - -(67) CometBroadcastHashJoin -Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Right output [2]: [d_date_sk#95, d_year#96] -Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight - -(68) CometProject -Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] - -(69) CometHashAggregate -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(70) CometExchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(71) CometColumnarToRow [codegen id : 3] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] - -(72) HashAggregate [codegen id : 3] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))#73] -Results [2]: [c_customer_id#84 AS customer_id#98, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))#73,18,2) AS year_total#99] - -(73) BroadcastExchange -Input [2]: [customer_id#98, year_total#99] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(74) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#98] -Join type: Inner -Join condition: (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) END > CASE WHEN (year_total#25 > 0.00) THEN (year_total#50 / year_total#25) END) - -(75) Project [codegen id : 4] -Output [1]: [customer_preferred_cust_flag#49] -Input [7]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, year_total#75, customer_id#98, year_total#99] - -(76) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#49] -Arguments: 100, [customer_preferred_cust_flag#49 ASC NULLS FIRST], [customer_preferred_cust_flag#49] +Input [12]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91, d_date_sk#93, d_year#94] +Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] + +(66) CometHashAggregate +Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] + +(67) CometExchange +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] +Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] + +(69) CometBroadcastExchange +Input [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#96, year_total#97] + +(70) CometBroadcastHashJoin +Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] +Right output [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#23], [customer_id#96], Inner, (CASE WHEN (year_total#73 > 0.00) THEN (year_total#97 / year_total#73) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#49 / year_total#24) END), BuildRight + +(71) CometProject +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#96, year_total#97] +Arguments: [customer_preferred_cust_flag#48], [customer_preferred_cust_flag#48] + +(72) CometTakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#48] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#48 ASC NULLS FIRST], output=[customer_preferred_cust_flag#48]), [customer_preferred_cust_flag#48], 100, 0, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] + +(73) CometColumnarToRow [codegen id : 1] +Input [1]: [customer_preferred_cust_flag#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (77) ++- * CometColumnarToRow (76) + +- CometFilter (75) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(74) 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 -(78) CometFilter +(75) 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)) -(79) CometColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(80) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -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=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#44 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometFilter (82) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometFilter (79) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(78) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) +(79) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(83) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#45, d_year#46] +(80) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#44, d_year#45] -(84) BroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(81) BroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +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#68 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#44 +Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#43 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt index fdab1c6f0d..781eae9054 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/extended.txt @@ -1,79 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#4)))#5,18,2) AS year_total#6)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#7 AS customer_id#8, c_preferred_cust_flag#9 AS customer_preferred_cust_flag#10, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#12)))#5,18,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#16 - ws_ext_discount_amt#17)))#18,18,2) AS year_total#19)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#20 AS customer_id#21, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#22 - ws_ext_discount_amt#23)))#18,18,2) AS year_total#24)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 @@ -91,4 +88,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 86 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt index 0e67122144..4599ecf539 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11.native_iceberg_compat/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (4) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - 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 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_preferred_cust_flag] + CometProject [customer_preferred_cust_flag] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,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] @@ -32,72 +32,60 @@ TakeOrderedAndProject [customer_preferred_cust_flag] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 + CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,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 + 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-v1_4-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt index b9d337c55b..b6edfa8800 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt @@ -1,80 +1,77 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Project (75) - +- * BroadcastHashJoin Inner BuildRight (74) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- * Project (38) - : : +- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (55) - : +- * Filter (54) - : +- * HashAggregate (53) - : +- * CometColumnarToRow (52) - : +- CometExchange (51) - : +- CometHashAggregate (50) - : +- CometProject (49) - : +- CometBroadcastHashJoin (48) - : :- CometProject (46) - : : +- CometBroadcastHashJoin (45) - : : :- CometProject (41) - : : : +- CometFilter (40) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (39) - : : +- CometBroadcastExchange (44) - : : +- CometFilter (43) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) - : +- ReusedExchange (47) - +- BroadcastExchange (73) - +- * HashAggregate (72) - +- * CometColumnarToRow (71) - +- CometExchange (70) - +- CometHashAggregate (69) - +- CometProject (68) - +- CometBroadcastHashJoin (67) - :- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (60) - : : +- CometFilter (59) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (58) - : +- CometBroadcastExchange (63) - : +- CometFilter (62) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (61) - +- ReusedExchange (66) +* CometColumnarToRow (73) ++- CometTakeOrderedAndProject (72) + +- CometProject (71) + +- CometBroadcastHashJoin (70) + :- CometProject (54) + : +- CometBroadcastHashJoin (53) + : :- CometProject (36) + : : +- 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 (52) + : +- CometFilter (51) + : +- CometHashAggregate (50) + : +- CometExchange (49) + : +- CometHashAggregate (48) + : +- CometProject (47) + : +- CometBroadcastHashJoin (46) + : :- CometProject (44) + : : +- CometBroadcastHashJoin (43) + : : :- CometProject (39) + : : : +- CometFilter (38) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + : : +- CometBroadcastExchange (42) + : : +- CometFilter (41) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (40) + : +- ReusedExchange (45) + +- CometBroadcastExchange (69) + +- CometHashAggregate (68) + +- CometExchange (67) + +- CometHashAggregate (66) + +- CometProject (65) + +- CometBroadcastHashJoin (64) + :- CometProject (62) + : +- CometBroadcastHashJoin (61) + : :- CometProject (57) + : : +- CometFilter (56) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (55) + : +- CometBroadcastExchange (60) + : +- CometFilter (59) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (58) + +- ReusedExchange (63) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer @@ -150,349 +147,329 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discoun 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 : 4] -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] - -(17) HashAggregate [codegen id : 4] +(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)))#23] -Results [2]: [c_customer_id#9 AS customer_id#24, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#23,18,2) AS year_total#25] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#24, year_total#25] -Condition : (isnotnull(year_total#25) AND (year_total#25 > 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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] +(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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) -(21) CometProject -Input [8]: [c_customer_sk#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Arguments: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#30, 1)) AS c_preferred_cust_flag#37, c_birth_country#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#32, 13)) AS c_login#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#33, 50)) AS c_email_address#39] +(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#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] 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#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Condition : isnotnull(ss_customer_sk#40) +(22) CometFilter +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Condition : isnotnull(ss_customer_sk#39) -(24) CometBroadcastExchange -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(25) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Right output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_sk#26], [ss_customer_sk#40], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight -(26) CometProject -Input [12]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(26) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: [d_date_sk#45, d_year#46] - -(30) CometBroadcastHashJoin -Left output [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Right output [2]: [d_date_sk#45, d_year#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#45], Inner, BuildRight - -(31) CometProject -Input [12]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43, d_date_sk#45, d_year#46] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] - -(32) CometHashAggregate -Input [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] - -(33) CometExchange -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] - -(35) HashAggregate [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23] -Results [3]: [c_customer_id#34 AS customer_id#48, c_preferred_cust_flag#37 AS customer_preferred_cust_flag#49, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23,18,2) AS year_total#50] - -(36) BroadcastExchange -Input [3]: [customer_id#48, customer_preferred_cust_flag#49, year_total#50] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#48] -Join type: Inner -Join condition: None - -(38) Project [codegen id : 4] -Output [4]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50] -Input [5]: [customer_id#24, year_total#25, customer_id#48, customer_preferred_cust_flag#49, year_total#50] - -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] +(27) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] + +(31) CometHashAggregate +Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(32) CometExchange +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(34) CometBroadcastExchange +Input [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [3]: [customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight + +(36) CometProject +Input [5]: [customer_id#23, year_total#24, customer_id#47, customer_preferred_cust_flag#48, year_total#49] +Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(40) CometFilter -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) +(38) CometFilter +Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Condition : (isnotnull(c_customer_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)))) -(41) CometProject -Input [8]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54, c_preferred_cust_flag#55, c_birth_country#56, c_login#57, c_email_address#58] -Arguments: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#55, 1)) AS c_preferred_cust_flag#62, c_birth_country#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#57, 13)) AS c_login#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#58, 50)) AS c_email_address#64] +(39) CometProject +Input [8]: [c_customer_sk#50, c_customer_id#51, c_first_name#52, c_last_name#53, c_preferred_cust_flag#54, c_birth_country#55, c_login#56, c_email_address#57] +Arguments: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63], [c_customer_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#51, 16)) AS c_customer_id#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#52, 20)) AS c_first_name#59, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#53, 30)) AS c_last_name#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#54, 1)) AS c_preferred_cust_flag#61, c_birth_country#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#56, 13)) AS c_login#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#57, 50)) AS c_email_address#63] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] 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#67), dynamicpruningexpression(ws_sold_date_sk#67 IN dynamicpruning#68)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(43) CometFilter -Input [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Condition : isnotnull(ws_bill_customer_sk#65) +(41) CometFilter +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Condition : isnotnull(ws_bill_customer_sk#64) -(44) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(42) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(45) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64] -Right output [4]: [ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [c_customer_sk#51], [ws_bill_customer_sk#65], Inner, BuildRight +(43) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63] +Right output [4]: [ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_sk#50], [ws_bill_customer_sk#64], Inner, BuildRight -(46) CometProject -Input [12]: [c_customer_sk#51, c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_bill_customer_sk#65, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] +(44) CometProject +Input [12]: [c_customer_sk#50, c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_bill_customer_sk#64, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] -(47) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#70, d_year#71] +(45) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#69, d_year#70] -(48) CometBroadcastHashJoin -Left output [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68] -Right output [2]: [d_date_sk#70, d_year#71] -Arguments: [ws_sold_date_sk#68], [d_date_sk#70], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67] +Right output [2]: [d_date_sk#69, d_year#70] +Arguments: [ws_sold_date_sk#67], [d_date_sk#69], Inner, BuildRight -(49) CometProject -Input [12]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, ws_sold_date_sk#68, d_date_sk#70, d_year#71] -Arguments: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71], [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71] +(47) CometProject +Input [12]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, ws_sold_date_sk#67, d_date_sk#69, d_year#70] +Arguments: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70], [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] + +(48) CometHashAggregate +Input [10]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, ws_ext_discount_amt#65, ws_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(49) CometExchange +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] +Arguments: hashpartitioning(c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (50) CometHashAggregate -Input [10]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, ws_ext_discount_amt#66, ws_ext_list_price#67, d_year#71] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))] - -(51) CometExchange -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] -Arguments: hashpartitioning(c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(52) CometColumnarToRow [codegen id : 2] -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] - -(53) HashAggregate [codegen id : 2] -Input [9]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71, sum#72] -Keys [8]: [c_customer_id#59, c_first_name#60, c_last_name#61, c_preferred_cust_flag#62, c_birth_country#56, c_login#63, c_email_address#64, d_year#71] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))#73] -Results [2]: [c_customer_id#59 AS customer_id#74, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#67 - ws_ext_discount_amt#66)))#73,18,2) AS year_total#75] - -(54) Filter [codegen id : 2] -Input [2]: [customer_id#74, year_total#75] -Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) - -(55) BroadcastExchange -Input [2]: [customer_id#74, year_total#75] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(56) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#74] -Join type: Inner -Join condition: None - -(57) Project [codegen id : 4] -Output [5]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, year_total#75] -Input [6]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, customer_id#74, year_total#75] - -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Input [9]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70, sum#71] +Keys [8]: [c_customer_id#58, c_first_name#59, c_last_name#60, c_preferred_cust_flag#61, c_birth_country#55, c_login#62, c_email_address#63, d_year#70] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#66 - ws_ext_discount_amt#65)))] + +(51) CometFilter +Input [2]: [customer_id#72, year_total#73] +Condition : (isnotnull(year_total#73) AND (year_total#73 > 0.00)) + +(52) CometBroadcastExchange +Input [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#72, year_total#73] + +(53) CometBroadcastHashJoin +Left output [4]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49] +Right output [2]: [customer_id#72, year_total#73] +Arguments: [customer_id#23], [customer_id#72], Inner, BuildRight + +(54) CometProject +Input [6]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, customer_id#72, year_total#73] +Arguments: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73], [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] + +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(59) CometFilter -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) +(56) CometFilter +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Condition : (isnotnull(c_customer_sk#74) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)))) -(60) CometProject -Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] -Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] +(57) CometProject +Input [8]: [c_customer_sk#74, c_customer_id#75, c_first_name#76, c_last_name#77, c_preferred_cust_flag#78, c_birth_country#79, c_login#80, c_email_address#81] +Arguments: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87], [c_customer_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#75, 16)) AS c_customer_id#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#76, 20)) AS c_first_name#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#77, 30)) AS c_last_name#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#78, 1)) AS c_preferred_cust_flag#85, c_birth_country#79, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#80, 13)) AS c_login#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#81, 50)) AS c_email_address#87] -(61) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] +PartitionFilters: [isnotnull(ws_sold_date_sk#91), dynamicpruningexpression(ws_sold_date_sk#91 IN dynamicpruning#92)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(62) CometFilter -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Condition : isnotnull(ws_bill_customer_sk#90) +(59) CometFilter +Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Condition : isnotnull(ws_bill_customer_sk#88) + +(60) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] + +(61) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87] +Right output [4]: [ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [c_customer_sk#74], [ws_bill_customer_sk#88], Inner, BuildRight + +(62) CometProject +Input [12]: [c_customer_sk#74, c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_bill_customer_sk#88, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] -(63) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +(63) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#93, d_year#94] (64) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] -Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight +Left output [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91] +Right output [2]: [d_date_sk#93, d_year#94] +Arguments: [ws_sold_date_sk#91], [d_date_sk#93], Inner, BuildRight (65) CometProject -Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] - -(66) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#95, d_year#96] - -(67) CometBroadcastHashJoin -Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] -Right output [2]: [d_date_sk#95, d_year#96] -Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight - -(68) CometProject -Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] -Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] - -(69) CometHashAggregate -Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] - -(70) CometExchange -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(71) CometColumnarToRow [codegen id : 3] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] - -(72) HashAggregate [codegen id : 3] -Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] -Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))#73] -Results [2]: [c_customer_id#84 AS customer_id#98, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))#73,18,2) AS year_total#99] - -(73) BroadcastExchange -Input [2]: [customer_id#98, year_total#99] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(74) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#98] -Join type: Inner -Join condition: (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) END > CASE WHEN (year_total#25 > 0.00) THEN (year_total#50 / year_total#25) END) - -(75) Project [codegen id : 4] -Output [1]: [customer_preferred_cust_flag#49] -Input [7]: [customer_id#24, year_total#25, customer_preferred_cust_flag#49, year_total#50, year_total#75, customer_id#98, year_total#99] - -(76) TakeOrderedAndProject -Input [1]: [customer_preferred_cust_flag#49] -Arguments: 100, [customer_preferred_cust_flag#49 ASC NULLS FIRST], [customer_preferred_cust_flag#49] +Input [12]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, ws_sold_date_sk#91, d_date_sk#93, d_year#94] +Arguments: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94], [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] + +(66) CometHashAggregate +Input [10]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, ws_ext_discount_amt#89, ws_ext_list_price#90, d_year#94] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] + +(67) CometExchange +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] +Arguments: hashpartitioning(c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [9]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94, sum#95] +Keys [8]: [c_customer_id#82, c_first_name#83, c_last_name#84, c_preferred_cust_flag#85, c_birth_country#79, c_login#86, c_email_address#87, d_year#94] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#90 - ws_ext_discount_amt#89)))] + +(69) CometBroadcastExchange +Input [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#96, year_total#97] + +(70) CometBroadcastHashJoin +Left output [5]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73] +Right output [2]: [customer_id#96, year_total#97] +Arguments: [customer_id#23], [customer_id#96], Inner, (CASE WHEN (year_total#73 > 0.00) THEN (year_total#97 / year_total#73) END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#49 / year_total#24) END), BuildRight + +(71) CometProject +Input [7]: [customer_id#23, year_total#24, customer_preferred_cust_flag#48, year_total#49, year_total#73, customer_id#96, year_total#97] +Arguments: [customer_preferred_cust_flag#48], [customer_preferred_cust_flag#48] + +(72) CometTakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#48] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_preferred_cust_flag#48 ASC NULLS FIRST], output=[customer_preferred_cust_flag#48]), [customer_preferred_cust_flag#48], 100, 0, [customer_preferred_cust_flag#48 ASC NULLS FIRST], [customer_preferred_cust_flag#48] + +(73) CometColumnarToRow [codegen id : 1] +Input [1]: [customer_preferred_cust_flag#48] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (77) ++- * CometColumnarToRow (76) + +- CometFilter (75) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (74) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(74) 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 -(78) CometFilter +(75) 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)) -(79) CometColumnarToRow [codegen id : 1] +(76) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(80) BroadcastExchange +(77) BroadcastExchange Input [2]: [d_date_sk#20, d_year#21] -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=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#44 -BroadcastExchange (84) -+- * CometColumnarToRow (83) - +- CometFilter (82) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (81) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometFilter (79) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (78) -(81) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(78) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(82) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) +(79) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(83) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#45, d_year#46] +(80) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#44, d_year#45] -(84) BroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(81) BroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +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#68 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#67 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 61 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#44 +Subquery:4 Hosting operator id = 58 Hosting Expression = ws_sold_date_sk#91 IN dynamicpruning#43 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt index fdab1c6f0d..781eae9054 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/extended.txt @@ -1,79 +1,76 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#4)))#5,18,2) AS year_total#6)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#7 AS customer_id#8, c_preferred_cust_flag#9 AS customer_preferred_cust_flag#10, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#12)))#5,18,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#16 - ws_ext_discount_amt#17)))#18,18,2) AS year_total#19)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#20 AS customer_id#21, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#22 - ws_ext_discount_amt#23)))#18,18,2) AS year_total#24)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 @@ -91,4 +88,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 86 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 80 out of 86 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt index 0e67122144..4599ecf539 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [customer_preferred_cust_flag] - WholeStageCodegen (4) - Project [customer_preferred_cust_flag] - BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] - BroadcastHashJoin [customer_id,customer_id] - Project [customer_id,year_total,customer_preferred_cust_flag,year_total] - 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 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_preferred_cust_flag] + CometProject [customer_preferred_cust_flag] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_preferred_cust_flag,year_total,customer_id,year_total] + CometProject [customer_id,year_total,customer_preferred_cust_flag,year_total] + CometBroadcastHashJoin [customer_id,year_total,customer_id,customer_preferred_cust_flag,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] @@ -32,72 +32,60 @@ TakeOrderedAndProject [customer_preferred_cust_flag] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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_preferred_cust_flag,year_total,sum] - CometColumnarToRow - InputAdapter - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + CometBroadcastExchange [customer_id,customer_preferred_cust_flag,year_total] #5 + CometHashAggregate [c_first_name,c_last_name,d_year,c_birth_country,c_login,c_email_address,sum] [customer_id,customer_preferred_cust_flag,year_total,c_customer_id,c_preferred_cust_flag,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 + 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-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt index 866cc628a9..8811d04e59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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_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 (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt index c467ccd3ea..3f41c97ff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt index 1fdffc3b73..fb83fd2f9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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 + 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-v1_4-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index 866cc628a9..8811d04e59 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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_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 (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt index c467ccd3ea..3f41c97ff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 1fdffc3b73..fb83fd2f9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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 + 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-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt index a4c1278d02..027d679419 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (34) -+- * CometColumnarToRow (33) +* CometColumnarToRow (34) ++- CometHashAggregate (33) +- CometExchange (32) +- CometHashAggregate (31) +- CometProject (30) @@ -188,15 +188,13 @@ Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sal Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(33) CometColumnarToRow [codegen id : 1] -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] - -(34) HashAggregate [codegen id : 1] +(33) CometHashAggregate Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#33, avg(UnscaledValue(ss_ext_sales_price#7))#34, avg(UnscaledValue(ss_ext_wholesale_cost#8))#35, sum(UnscaledValue(ss_ext_wholesale_cost#8))#36] -Results [4]: [avg(ss_quantity#5)#33 AS avg(ss_quantity)#37, cast((avg(UnscaledValue(ss_ext_sales_price#7))#34 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#38, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#35 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#39, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#36,17,2) AS sum(ss_ext_wholesale_cost)#40] + +(34) CometColumnarToRow [codegen id : 1] +Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt index f74c20954d..08e9beb692 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(avg(ss_quantity#1)#2 AS avg(ss_quantity)#3, cast((avg(UnscaledValue(ss_ext_sales_price#4))#5 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#6, cast((avg(UnscaledValue(ss_ext_wholesale_cost#7))#8 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#9, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#7))#10,17,2) AS sum(ss_ext_wholesale_cost)#11)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt index f7cfb4ea09..8ef882a435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt index a4c1278d02..027d679419 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (34) -+- * CometColumnarToRow (33) +* CometColumnarToRow (34) ++- CometHashAggregate (33) +- CometExchange (32) +- CometHashAggregate (31) +- CometProject (30) @@ -188,15 +188,13 @@ Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sal Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(33) CometColumnarToRow [codegen id : 1] -Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] - -(34) HashAggregate [codegen id : 1] +(33) CometHashAggregate Input [7]: [sum#26, count#27, sum#28, count#29, sum#30, count#31, sum#32] Keys: [] Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] -Aggregate Attributes [4]: [avg(ss_quantity#5)#33, avg(UnscaledValue(ss_ext_sales_price#7))#34, avg(UnscaledValue(ss_ext_wholesale_cost#8))#35, sum(UnscaledValue(ss_ext_wholesale_cost#8))#36] -Results [4]: [avg(ss_quantity#5)#33 AS avg(ss_quantity)#37, cast((avg(UnscaledValue(ss_ext_sales_price#7))#34 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#38, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#35 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#39, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#36,17,2) AS sum(ss_ext_wholesale_cost)#40] + +(34) CometColumnarToRow [codegen id : 1] +Input [4]: [avg(ss_quantity)#33, avg(ss_ext_sales_price)#34, avg(ss_ext_wholesale_cost)#35, sum(ss_ext_wholesale_cost)#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt index f74c20954d..08e9beb692 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(avg(ss_quantity#1)#2 AS avg(ss_quantity)#3, cast((avg(UnscaledValue(ss_ext_sales_price#4))#5 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#6, cast((avg(UnscaledValue(ss_ext_wholesale_cost#7))#8 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#9, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#7))#10,17,2) AS sum(ss_ext_wholesale_cost)#11)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 38 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt index f7cfb4ea09..8ef882a435 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost))] CometExchange #1 CometHashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum] CometProject [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt index 4527be4e9d..8a4fb53ece 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) +- CometExchange (21) +- CometHashAggregate (20) +- CometProject (19) @@ -126,19 +126,17 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Input [2]: [ca_zip#11, sum#15] Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum#15] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [2]: [ca_zip#11, sum#15] Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#16] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17,2) AS sum(cs_sales_price)#17] -(24) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#17] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#17] +(23) CometTakeOrderedAndProject +Input [2]: [ca_zip#11, sum(cs_sales_price)#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] + +(24) CometColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#11, sum(cs_sales_price)#16] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt index d99c44f9e1..6de0c64850 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_zip#1, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#3,17,2) AS sum(cs_sales_price)#4)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt index 6c4d568d44..c39b96efe3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (1) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] CometExchange [ca_zip] #1 CometHashAggregate [cs_sales_price] [ca_zip,sum] CometProject [cs_sales_price,ca_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt index 4527be4e9d..8a4fb53ece 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * HashAggregate (23) - +- * CometColumnarToRow (22) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometHashAggregate (22) +- CometExchange (21) +- CometHashAggregate (20) +- CometProject (19) @@ -126,19 +126,17 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] Input [2]: [ca_zip#11, sum#15] Arguments: hashpartitioning(ca_zip#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [2]: [ca_zip#11, sum#15] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [2]: [ca_zip#11, sum#15] Keys [1]: [ca_zip#11] Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#16] -Results [2]: [ca_zip#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#16,17,2) AS sum(cs_sales_price)#17] -(24) TakeOrderedAndProject -Input [2]: [ca_zip#11, sum(cs_sales_price)#17] -Arguments: 100, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#17] +(23) CometTakeOrderedAndProject +Input [2]: [ca_zip#11, sum(cs_sales_price)#16] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ca_zip#11 ASC NULLS FIRST], output=[ca_zip#11,sum(cs_sales_price)#16]), [ca_zip#11, sum(cs_sales_price)#16], 100, 0, [ca_zip#11 ASC NULLS FIRST], [ca_zip#11, sum(cs_sales_price)#16] + +(24) CometColumnarToRow [codegen id : 1] +Input [2]: [ca_zip#11, sum(cs_sales_price)#16] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt index d99c44f9e1..6de0c64850 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_zip#1, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#3,17,2) AS sum(cs_sales_price)#4)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -29,4 +29,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt index 6c4d568d44..c39b96efe3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] - WholeStageCodegen (1) - HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + CometHashAggregate [sum] [ca_zip,sum(cs_sales_price),sum(UnscaledValue(cs_sales_price))] CometExchange [ca_zip] #1 CometHashAggregate [cs_sales_price] [ca_zip,sum] CometProject [cs_sales_price,ca_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt index 678ae3c7c3..7a4c0fa400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (41) -+- * CometColumnarToRow (40) +* CometColumnarToRow (41) ++- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) @@ -225,13 +225,11 @@ Results [3]: [sum#20, sum#21, count#25] Input [3]: [sum#20, sum#21, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#20, sum#21, count#25] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#22,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#23,17,2) AS total net profit #28] + +(41) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt index c6125cb268..2ad029e444 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(cs_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -40,4 +40,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 39 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt index c703a75e78..8427aa49a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt index 678ae3c7c3..7a4c0fa400 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (41) -+- * CometColumnarToRow (40) +* CometColumnarToRow (41) ++- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) @@ -225,13 +225,11 @@ Results [3]: [sum#20, sum#21, count#25] Input [3]: [sum#20, sum#21, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#20, sum#21, count#25] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#22, sum(UnscaledValue(cs_net_profit#7))#23, count(cs_order_number#5)#24] -Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#22,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#23,17,2) AS total net profit #28] + +(41) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt index c6125cb268..2ad029e444 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(cs_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -40,4 +40,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 36 out of 39 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt index c703a75e78..8427aa49a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(cs_order_number),sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt index bab37e51d2..056ed8dd1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- * CometColumnarToRow (34) +* CometColumnarToRow (36) ++- CometTakeOrderedAndProject (35) + +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) +- CometProject (31) @@ -194,17 +194,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] - -(35) HashAggregate [codegen id : 1] +(34) CometHashAggregate Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#26] -Results [5]: [i_brand_id#10 AS brand_id#27, i_brand#15 AS brand#28, i_manufact_id#12, i_manufact#16, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#26,17,2) AS ext_price#29] -(36) TakeOrderedAndProject -Input [5]: [brand_id#27, brand#28, i_manufact_id#12, i_manufact#16, ext_price#29] -Arguments: 100, [ext_price#29 DESC NULLS LAST, brand#28 ASC NULLS FIRST, brand_id#27 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#27, brand#28, i_manufact_id#12, i_manufact#16, ext_price#29] +(35) CometTakeOrderedAndProject +Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] + +(36) CometColumnarToRow [codegen id : 1] +Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt index 68fdff2831..a8cfde2d31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, i_manufact_id#5, i_manufact#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#8,17,2) AS ext_price#9)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt index 9b585fcb77..93ab89c142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] + CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt index bab37e51d2..056ed8dd1f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (36) -+- * HashAggregate (35) - +- * CometColumnarToRow (34) +* CometColumnarToRow (36) ++- CometTakeOrderedAndProject (35) + +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) +- CometProject (31) @@ -194,17 +194,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] Arguments: hashpartitioning(i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] - -(35) HashAggregate [codegen id : 1] +(34) CometHashAggregate Input [5]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16, sum#25] Keys [4]: [i_brand#15, i_brand_id#10, i_manufact_id#12, i_manufact#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#26] -Results [5]: [i_brand_id#10 AS brand_id#27, i_brand#15 AS brand#28, i_manufact_id#12, i_manufact#16, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#26,17,2) AS ext_price#29] -(36) TakeOrderedAndProject -Input [5]: [brand_id#27, brand#28, i_manufact_id#12, i_manufact#16, ext_price#29] -Arguments: 100, [ext_price#29 DESC NULLS LAST, brand#28 ASC NULLS FIRST, brand_id#27 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#27, brand#28, i_manufact_id#12, i_manufact#16, ext_price#29] +(35) CometTakeOrderedAndProject +Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#28 DESC NULLS LAST,brand#27 ASC NULLS FIRST,brand_id#26 ASC NULLS FIRST,i_manufact_id#12 ASC NULLS FIRST,i_manufact#16 ASC NULLS FIRST], output=[brand_id#26,brand#27,i_manufact_id#12,i_manufact#16,ext_price#28]), [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28], 100, 0, [ext_price#28 DESC NULLS LAST, brand#27 ASC NULLS FIRST, brand_id#26 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#16 ASC NULLS FIRST], [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] + +(36) CometColumnarToRow [codegen id : 1] +Input [5]: [brand_id#26, brand#27, i_manufact_id#12, i_manufact#16, ext_price#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt index 68fdff2831..a8cfde2d31 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, i_manufact_id#5, i_manufact#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#8,17,2) AS ext_price#9)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -35,4 +35,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 35 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt index 9b585fcb77..93ab89c142 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,i_manufact_id,i_manufact,ext_price] + CometHashAggregate [sum] [brand_id,brand,i_manufact_id,i_manufact,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] CometProject [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt index f6648cdf58..a7a7ef7b7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/explain.txt @@ -1,42 +1,38 @@ == Physical Plan == -* CometColumnarToRow (38) -+- CometSort (37) - +- CometColumnarExchange (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * HashAggregate (15) - : : +- * CometColumnarToRow (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - +- BroadcastExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * HashAggregate (25) - : +- * CometColumnarToRow (24) - : +- ReusedExchange (23) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +* CometColumnarToRow (34) ++- CometSort (33) + +- CometExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometHashAggregate (14) + : : +- CometExchange (13) + : : +- CometHashAggregate (12) + : : +- CometProject (11) + : : +- CometBroadcastHashJoin (10) + : : :- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) + : : +- CometBroadcastExchange (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometHashAggregate (22) + : +- ReusedExchange (21) + +- CometBroadcastExchange (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -102,115 +98,96 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(14) CometColumnarToRow [codegen id : 4] -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] - -(15) HashAggregate [codegen id : 4] +(14) CometHashAggregate Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))#26] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END))#20,17,2) AS sun_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END))#21,17,2) AS mon_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END))#22,17,2) AS tue_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END))#23,17,2) AS wed_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END))#24,17,2) AS thu_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END))#25,17,2) AS fri_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))#26,17,2) AS sat_sales#33] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#34, d_year#35] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#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_week_seq)] ReadSchema: struct -(17) CometFilter -Input [2]: [d_week_seq#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2001)) AND isnotnull(d_week_seq#34)) - -(18) CometProject -Input [2]: [d_week_seq#34, d_year#35] -Arguments: [d_week_seq#34], [d_week_seq#34] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#34] +(16) CometFilter +Input [2]: [d_week_seq#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) -(20) BroadcastExchange -Input [1]: [d_week_seq#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometProject +Input [2]: [d_week_seq#20, d_year#21] +Arguments: [d_week_seq#20], [d_week_seq#20] -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#34] -Join type: Inner -Join condition: None +(18) CometBroadcastExchange +Input [1]: [d_week_seq#20] +Arguments: [d_week_seq#20] -(22) Project [codegen id : 4] -Output [8]: [d_week_seq#10 AS d_week_seq1#36, sun_sales#27 AS sun_sales1#37, mon_sales#28 AS mon_sales1#38, tue_sales#29 AS tue_sales1#39, wed_sales#30 AS wed_sales1#40, thu_sales#31 AS thu_sales1#41, fri_sales#32 AS fri_sales1#42, sat_sales#33 AS sat_sales1#43] -Input [9]: [d_week_seq#10, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33, d_week_seq#34] +(19) CometBroadcastHashJoin +Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] +Right output [1]: [d_week_seq#20] +Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight -(23) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] +(20) CometProject +Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] +Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] -(24) CometColumnarToRow [codegen id : 3] -Input [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] +(21) ReusedExchange [Reuses operator id: 13] +Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -(25) HashAggregate [codegen id : 3] -Input [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] -Keys [1]: [d_week_seq#44] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))#26] -Results [8]: [d_week_seq#44, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END))#20,17,2) AS sun_sales#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END))#21,17,2) AS mon_sales#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END))#22,17,2) AS tue_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END))#23,17,2) AS wed_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END))#24,17,2) AS thu_sales#58, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END))#25,17,2) AS fri_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))#26,17,2) AS sat_sales#60] +(22) CometHashAggregate +Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] +Keys [1]: [d_week_seq#37] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN sales_price#46 END))] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#61, d_year#62] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#47, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_week_seq#61, d_year#62] -Condition : ((isnotnull(d_year#62) AND (d_year#62 = 2002)) AND isnotnull(d_week_seq#61)) +(24) CometFilter +Input [2]: [d_week_seq#47, d_year#48] +Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_week_seq#47)) + +(25) CometProject +Input [2]: [d_week_seq#47, d_year#48] +Arguments: [d_week_seq#47], [d_week_seq#47] + +(26) CometBroadcastExchange +Input [1]: [d_week_seq#47] +Arguments: [d_week_seq#47] + +(27) CometBroadcastHashJoin +Left output [8]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] +Right output [1]: [d_week_seq#47] +Arguments: [d_week_seq#37], [d_week_seq#47], Inner, BuildRight (28) CometProject -Input [2]: [d_week_seq#61, d_year#62] -Arguments: [d_week_seq#61], [d_week_seq#61] - -(29) CometColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#61] - -(30) BroadcastExchange -Input [1]: [d_week_seq#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#44] -Right keys [1]: [d_week_seq#61] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 3] -Output [8]: [d_week_seq#44 AS d_week_seq2#63, sun_sales#54 AS sun_sales2#64, mon_sales#55 AS mon_sales2#65, tue_sales#56 AS tue_sales2#66, wed_sales#57 AS wed_sales2#67, thu_sales#58 AS thu_sales2#68, fri_sales#59 AS fri_sales2#69, sat_sales#60 AS sat_sales2#70] -Input [9]: [d_week_seq#44, sun_sales#54, mon_sales#55, tue_sales#56, wed_sales#57, thu_sales#58, fri_sales#59, sat_sales#60, d_week_seq#61] - -(33) BroadcastExchange -Input [8]: [d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_week_seq1#36] -Right keys [1]: [(d_week_seq2#63 - 53)] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 4] -Output [8]: [d_week_seq1#36, round((sun_sales1#37 / sun_sales2#64), 2) AS round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1#38 / mon_sales2#65), 2) AS round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1#39 / tue_sales2#66), 2) AS round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1#40 / wed_sales2#67), 2) AS round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1#41 / thu_sales2#68), 2) AS round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1#42 / fri_sales2#69), 2) AS round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1#43 / sat_sales2#70), 2) AS round((sat_sales1 / sat_sales2), 2)#77] -Input [16]: [d_week_seq1#36, sun_sales1#37, mon_sales1#38, tue_sales1#39, wed_sales1#40, thu_sales1#41, fri_sales1#42, sat_sales1#43, d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] - -(36) CometColumnarExchange -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] -Arguments: rangepartitioning(d_week_seq1#36 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometSort -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] -Arguments: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77], [d_week_seq1#36 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] +Input [9]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, d_week_seq#47] +Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63], [d_week_seq#37 AS d_week_seq2#56, sun_sales#49 AS sun_sales2#57, mon_sales#50 AS mon_sales2#58, tue_sales#51 AS tue_sales2#59, wed_sales#52 AS wed_sales2#60, thu_sales#53 AS thu_sales2#61, fri_sales#54 AS fri_sales2#62, sat_sales#55 AS sat_sales2#63] + +(29) CometBroadcastExchange +Input [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] + +(30) CometBroadcastHashJoin +Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] +Right output [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq1#29], [(d_week_seq2#56 - 53)], Inner, BuildRight + +(31) CometProject +Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#57), 2) AS round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1#31 / mon_sales2#58), 2) AS round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1#32 / tue_sales2#59), 2) AS round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1#33 / wed_sales2#60), 2) AS round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1#34 / thu_sales2#61), 2) AS round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1#35 / fri_sales2#62), 2) AS round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1#36 / sat_sales2#63), 2) AS round((sat_sales1 / sat_sales2), 2)#70] + +(32) CometExchange +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometSort +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] +Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] + +(34) CometColumnarToRow [codegen id : 1] +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt index 81b96b4184..15f5db847d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/extended.txt @@ -1,52 +1,48 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Sunday ) THEN sales_price#3 END))#4,17,2) AS sun_sales#5, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Monday ) THEN sales_price#3 END))#6,17,2) AS mon_sales#7, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Tuesday ) THEN sales_price#3 END))#8,17,2) AS tue_sales#9, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Wednesday) THEN sales_price#3 END))#10,17,2) AS wed_sales#11, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Thursday ) THEN sales_price#3 END))#12,17,2) AS thu_sales#13, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Friday ) THEN sales_price#3 END))#14,17,2) AS fri_sales#15, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Saturday ) THEN sales_price#3 END))#16,17,2) AS sat_sales#17)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- 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.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#18, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Sunday ) THEN sales_price#20 END))#4,17,2) AS sun_sales#21, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Monday ) THEN sales_price#20 END))#6,17,2) AS mon_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Tuesday ) THEN sales_price#20 END))#8,17,2) AS tue_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Wednesday) THEN sales_price#20 END))#10,17,2) AS wed_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Thursday ) THEN sales_price#20 END))#12,17,2) AS thu_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Friday ) THEN sales_price#20 END))#14,17,2) AS fri_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Saturday ) THEN sales_price#20 END))#16,17,2) AS sat_sales#27)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- 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.date_dim + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- CometProject + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- CometProject + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.date_dim -Comet accelerated 34 out of 45 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt index 150fb01644..852c5fca0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2.native_iceberg_compat/simplified.txt @@ -1,51 +1,36 @@ -WholeStageCodegen (5) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometColumnarExchange [d_week_seq1] #1 - WholeStageCodegen (4) - Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometExchange [d_week_seq1] #1 + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometExchange [d_week_seq] #2 + CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + CometBroadcastExchange [d_week_seq] #6 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt index f6648cdf58..a7a7ef7b7c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt @@ -1,42 +1,38 @@ == Physical Plan == -* CometColumnarToRow (38) -+- CometSort (37) - +- CometColumnarExchange (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (22) - : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * HashAggregate (15) - : : +- * CometColumnarToRow (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometUnion (5) - : : : :- CometProject (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometProject (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- BroadcastExchange (20) - : +- * CometColumnarToRow (19) - : +- CometProject (18) - : +- CometFilter (17) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (16) - +- BroadcastExchange (33) - +- * Project (32) - +- * BroadcastHashJoin Inner BuildRight (31) - :- * HashAggregate (25) - : +- * CometColumnarToRow (24) - : +- ReusedExchange (23) - +- BroadcastExchange (30) - +- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +* CometColumnarToRow (34) ++- CometSort (33) + +- CometExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometHashAggregate (14) + : : +- CometExchange (13) + : : +- CometHashAggregate (12) + : : +- CometProject (11) + : : +- CometBroadcastHashJoin (10) + : : :- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (3) + : : +- CometBroadcastExchange (9) + : : +- CometProject (8) + : : +- CometFilter (7) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (15) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometHashAggregate (22) + : +- ReusedExchange (21) + +- CometBroadcastExchange (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (23) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -102,115 +98,96 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(14) CometColumnarToRow [codegen id : 4] -Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] - -(15) HashAggregate [codegen id : 4] +(14) CometHashAggregate Input [8]: [d_week_seq#10, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))#26] -Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Sunday ) THEN sales_price#4 END))#20,17,2) AS sun_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Monday ) THEN sales_price#4 END))#21,17,2) AS mon_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Tuesday ) THEN sales_price#4 END))#22,17,2) AS tue_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Wednesday) THEN sales_price#4 END))#23,17,2) AS wed_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Thursday ) THEN sales_price#4 END))#24,17,2) AS thu_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Friday ) THEN sales_price#4 END))#25,17,2) AS fri_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#12 = Saturday ) THEN sales_price#4 END))#26,17,2) AS sat_sales#33] -(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#34, d_year#35] +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#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_week_seq)] ReadSchema: struct -(17) CometFilter -Input [2]: [d_week_seq#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2001)) AND isnotnull(d_week_seq#34)) - -(18) CometProject -Input [2]: [d_week_seq#34, d_year#35] -Arguments: [d_week_seq#34], [d_week_seq#34] - -(19) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#34] +(16) CometFilter +Input [2]: [d_week_seq#20, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_week_seq#20)) -(20) BroadcastExchange -Input [1]: [d_week_seq#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(17) CometProject +Input [2]: [d_week_seq#20, d_year#21] +Arguments: [d_week_seq#20], [d_week_seq#20] -(21) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_week_seq#10] -Right keys [1]: [d_week_seq#34] -Join type: Inner -Join condition: None +(18) CometBroadcastExchange +Input [1]: [d_week_seq#20] +Arguments: [d_week_seq#20] -(22) Project [codegen id : 4] -Output [8]: [d_week_seq#10 AS d_week_seq1#36, sun_sales#27 AS sun_sales1#37, mon_sales#28 AS mon_sales1#38, tue_sales#29 AS tue_sales1#39, wed_sales#30 AS wed_sales1#40, thu_sales#31 AS thu_sales1#41, fri_sales#32 AS fri_sales1#42, sat_sales#33 AS sat_sales1#43] -Input [9]: [d_week_seq#10, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33, d_week_seq#34] +(19) CometBroadcastHashJoin +Left output [8]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28] +Right output [1]: [d_week_seq#20] +Arguments: [d_week_seq#10], [d_week_seq#20], Inner, BuildRight -(23) ReusedExchange [Reuses operator id: 13] -Output [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] +(20) CometProject +Input [9]: [d_week_seq#10, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, d_week_seq#20] +Arguments: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36], [d_week_seq#10 AS d_week_seq1#29, sun_sales#22 AS sun_sales1#30, mon_sales#23 AS mon_sales1#31, tue_sales#24 AS tue_sales1#32, wed_sales#25 AS wed_sales1#33, thu_sales#26 AS thu_sales1#34, fri_sales#27 AS fri_sales1#35, sat_sales#28 AS sat_sales1#36] -(24) CometColumnarToRow [codegen id : 3] -Input [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] +(21) ReusedExchange [Reuses operator id: 13] +Output [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] -(25) HashAggregate [codegen id : 3] -Input [8]: [d_week_seq#44, sum#45, sum#46, sum#47, sum#48, sum#49, sum#50, sum#51] -Keys [1]: [d_week_seq#44] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END)), sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))#26] -Results [8]: [d_week_seq#44, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Sunday ) THEN sales_price#53 END))#20,17,2) AS sun_sales#54, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Monday ) THEN sales_price#53 END))#21,17,2) AS mon_sales#55, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Tuesday ) THEN sales_price#53 END))#22,17,2) AS tue_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Wednesday) THEN sales_price#53 END))#23,17,2) AS wed_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Thursday ) THEN sales_price#53 END))#24,17,2) AS thu_sales#58, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Friday ) THEN sales_price#53 END))#25,17,2) AS fri_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#52 = Saturday ) THEN sales_price#53 END))#26,17,2) AS sat_sales#60] +(22) CometHashAggregate +Input [8]: [d_week_seq#37, sum#38, sum#39, sum#40, sum#41, sum#42, sum#43, sum#44] +Keys [1]: [d_week_seq#37] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#45 = Sunday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Monday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Tuesday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Wednesday) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Thursday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Friday ) THEN sales_price#46 END)), sum(UnscaledValue(CASE WHEN (d_day_name#45 = Saturday ) THEN sales_price#46 END))] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_week_seq#61, d_year#62] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#47, d_year#48] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(27) CometFilter -Input [2]: [d_week_seq#61, d_year#62] -Condition : ((isnotnull(d_year#62) AND (d_year#62 = 2002)) AND isnotnull(d_week_seq#61)) +(24) CometFilter +Input [2]: [d_week_seq#47, d_year#48] +Condition : ((isnotnull(d_year#48) AND (d_year#48 = 2002)) AND isnotnull(d_week_seq#47)) + +(25) CometProject +Input [2]: [d_week_seq#47, d_year#48] +Arguments: [d_week_seq#47], [d_week_seq#47] + +(26) CometBroadcastExchange +Input [1]: [d_week_seq#47] +Arguments: [d_week_seq#47] + +(27) CometBroadcastHashJoin +Left output [8]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55] +Right output [1]: [d_week_seq#47] +Arguments: [d_week_seq#37], [d_week_seq#47], Inner, BuildRight (28) CometProject -Input [2]: [d_week_seq#61, d_year#62] -Arguments: [d_week_seq#61], [d_week_seq#61] - -(29) CometColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#61] - -(30) BroadcastExchange -Input [1]: [d_week_seq#61] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_week_seq#44] -Right keys [1]: [d_week_seq#61] -Join type: Inner -Join condition: None - -(32) Project [codegen id : 3] -Output [8]: [d_week_seq#44 AS d_week_seq2#63, sun_sales#54 AS sun_sales2#64, mon_sales#55 AS mon_sales2#65, tue_sales#56 AS tue_sales2#66, wed_sales#57 AS wed_sales2#67, thu_sales#58 AS thu_sales2#68, fri_sales#59 AS fri_sales2#69, sat_sales#60 AS sat_sales2#70] -Input [9]: [d_week_seq#44, sun_sales#54, mon_sales#55, tue_sales#56, wed_sales#57, thu_sales#58, fri_sales#59, sat_sales#60, d_week_seq#61] - -(33) BroadcastExchange -Input [8]: [d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] -Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=4] - -(34) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [d_week_seq1#36] -Right keys [1]: [(d_week_seq2#63 - 53)] -Join type: Inner -Join condition: None - -(35) Project [codegen id : 4] -Output [8]: [d_week_seq1#36, round((sun_sales1#37 / sun_sales2#64), 2) AS round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1#38 / mon_sales2#65), 2) AS round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1#39 / tue_sales2#66), 2) AS round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1#40 / wed_sales2#67), 2) AS round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1#41 / thu_sales2#68), 2) AS round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1#42 / fri_sales2#69), 2) AS round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1#43 / sat_sales2#70), 2) AS round((sat_sales1 / sat_sales2), 2)#77] -Input [16]: [d_week_seq1#36, sun_sales1#37, mon_sales1#38, tue_sales1#39, wed_sales1#40, thu_sales1#41, fri_sales1#42, sat_sales1#43, d_week_seq2#63, sun_sales2#64, mon_sales2#65, tue_sales2#66, wed_sales2#67, thu_sales2#68, fri_sales2#69, sat_sales2#70] - -(36) CometColumnarExchange -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] -Arguments: rangepartitioning(d_week_seq1#36 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(37) CometSort -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] -Arguments: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77], [d_week_seq1#36 ASC NULLS FIRST] - -(38) CometColumnarToRow [codegen id : 5] -Input [8]: [d_week_seq1#36, round((sun_sales1 / sun_sales2), 2)#71, round((mon_sales1 / mon_sales2), 2)#72, round((tue_sales1 / tue_sales2), 2)#73, round((wed_sales1 / wed_sales2), 2)#74, round((thu_sales1 / thu_sales2), 2)#75, round((fri_sales1 / fri_sales2), 2)#76, round((sat_sales1 / sat_sales2), 2)#77] +Input [9]: [d_week_seq#37, sun_sales#49, mon_sales#50, tue_sales#51, wed_sales#52, thu_sales#53, fri_sales#54, sat_sales#55, d_week_seq#47] +Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63], [d_week_seq#37 AS d_week_seq2#56, sun_sales#49 AS sun_sales2#57, mon_sales#50 AS mon_sales2#58, tue_sales#51 AS tue_sales2#59, wed_sales#52 AS wed_sales2#60, thu_sales#53 AS thu_sales2#61, fri_sales#54 AS fri_sales2#62, sat_sales#55 AS sat_sales2#63] + +(29) CometBroadcastExchange +Input [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] + +(30) CometBroadcastHashJoin +Left output [8]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36] +Right output [8]: [d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq1#29], [(d_week_seq2#56 - 53)], Inner, BuildRight + +(31) CometProject +Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#56, sun_sales2#57, mon_sales2#58, tue_sales2#59, wed_sales2#60, thu_sales2#61, fri_sales2#62, sat_sales2#63] +Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#57), 2) AS round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1#31 / mon_sales2#58), 2) AS round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1#32 / tue_sales2#59), 2) AS round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1#33 / wed_sales2#60), 2) AS round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1#34 / thu_sales2#61), 2) AS round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1#35 / fri_sales2#62), 2) AS round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1#36 / sat_sales2#63), 2) AS round((sat_sales1 / sat_sales2), 2)#70] + +(32) CometExchange +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometSort +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] +Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70], [d_week_seq1#29 ASC NULLS FIRST] + +(34) CometColumnarToRow [codegen id : 1] +Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#64, round((mon_sales1 / mon_sales2), 2)#65, round((tue_sales1 / tue_sales2), 2)#66, round((wed_sales1 / wed_sales2), 2)#67, round((thu_sales1 / thu_sales2), 2)#68, round((fri_sales1 / fri_sales2), 2)#69, round((sat_sales1 / sat_sales2), 2)#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt index 81b96b4184..15f5db847d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/extended.txt @@ -1,52 +1,48 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Sunday ) THEN sales_price#3 END))#4,17,2) AS sun_sales#5, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Monday ) THEN sales_price#3 END))#6,17,2) AS mon_sales#7, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Tuesday ) THEN sales_price#3 END))#8,17,2) AS tue_sales#9, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Wednesday) THEN sales_price#3 END))#10,17,2) AS wed_sales#11, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Thursday ) THEN sales_price#3 END))#12,17,2) AS thu_sales#13, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Friday ) THEN sales_price#3 END))#14,17,2) AS fri_sales#15, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#2 = Saturday ) THEN sales_price#3 END))#16,17,2) AS sat_sales#17)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometUnion - : : : :- CometProject - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : +- 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.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#18, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Sunday ) THEN sales_price#20 END))#4,17,2) AS sun_sales#21, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Monday ) THEN sales_price#20 END))#6,17,2) AS mon_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Tuesday ) THEN sales_price#20 END))#8,17,2) AS tue_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Wednesday) THEN sales_price#20 END))#10,17,2) AS wed_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Thursday ) THEN sales_price#20 END))#12,17,2) AS thu_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Friday ) THEN sales_price#20 END))#14,17,2) AS fri_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#19 = Saturday ) THEN sales_price#20 END))#16,17,2) AS sat_sales#27)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometUnion - : : :- CometProject - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- CometProject - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : +- 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.date_dim + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometUnion + : : : :- CometProject + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- CometProject + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometUnion + : : :- CometProject + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- CometProject + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_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.date_dim -Comet accelerated 34 out of 45 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 45 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt index 150fb01644..852c5fca0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -1,51 +1,36 @@ -WholeStageCodegen (5) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometColumnarExchange [d_week_seq1] #1 - WholeStageCodegen (4) - Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [d_week_seq1,d_week_seq2] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_week_seq] #2 - CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] - CometProject [sales_price,d_week_seq,d_day_name] - CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] - CometUnion [sold_date_sk,sales_price] - CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] - CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] - CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 - CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] - CometFilter [d_date_sk,d_week_seq,d_day_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_week_seq,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometExchange [d_week_seq1] #1 + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] + CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + CometExchange [d_week_seq] #2 + CometHashAggregate [d_day_name,sales_price] [d_week_seq,sum,sum,sum,sum,sum,sum,sum] + CometProject [sales_price,d_week_seq,d_day_name] + CometBroadcastHashJoin [sold_date_sk,sales_price,d_date_sk,d_week_seq,d_day_name] + CometUnion [sold_date_sk,sales_price] + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + CometBroadcastExchange [d_date_sk,d_week_seq,d_day_name] #3 + CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] + CometFilter [d_date_sk,d_week_seq,d_day_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] + CometBroadcastExchange [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,d_week_seq] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END))] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + CometBroadcastExchange [d_week_seq] #6 + CometProject [d_week_seq] + CometFilter [d_week_seq,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt index 88752d6e12..e6612e7574 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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_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 (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt index a2d8b16a2c..cd52b2cd12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt index db00a01c2d..2958d060fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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 + 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-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index 88752d6e12..e6612e7574 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) TakeOrderedAndProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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_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 (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt index a2d8b16a2c..cd52b2cd12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index db00a01c2d..2958d060fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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 + 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-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt index f2c8b8a999..f076c7886b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) +- CometProject (39) @@ -240,19 +240,17 @@ Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(Unscale Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(42) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] - -(43) HashAggregate [codegen id : 1] +(42) CometHashAggregate Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#37, sum(UnscaledValue(sr_net_loss#11))#38, sum(UnscaledValue(cs_net_profit#16))#39] -Results [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#37,17,2) AS store_sales_profit#40, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#38,17,2) AS store_returns_loss#41, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#39,17,2) AS catalog_sales_profit#42] -(44) TakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] -Arguments: 100, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] +(43) CometTakeOrderedAndProject +Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] + +(44) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt index cded28ce81..4e1a4f3ec0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, s_store_id#3, s_store_name#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS store_sales_profit#7, MakeDecimal(sum(UnscaledValue(sr_net_loss#8))#9,17,2) AS store_returns_loss#10, MakeDecimal(sum(UnscaledValue(cs_net_profit#11))#12,17,2) AS catalog_sales_profit#13)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -59,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 50 out of 57 eligible operators (87%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt index 2140d72398..dcad304452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt index f2c8b8a999..f076c7886b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) +- CometProject (39) @@ -240,19 +240,17 @@ Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(Unscale Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] Arguments: hashpartitioning(i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(42) CometColumnarToRow [codegen id : 1] -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] - -(43) HashAggregate [codegen id : 1] +(42) CometHashAggregate Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, sum#34, sum#35, sum#36] Keys [4]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28] Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#37, sum(UnscaledValue(sr_net_loss#11))#38, sum(UnscaledValue(cs_net_profit#16))#39] -Results [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#37,17,2) AS store_sales_profit#40, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#38,17,2) AS store_returns_loss#41, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#39,17,2) AS catalog_sales_profit#42] -(44) TakeOrderedAndProject -Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] -Arguments: 100, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#40, store_returns_loss#41, catalog_sales_profit#42] +(43) CometTakeOrderedAndProject +Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#33 ASC NULLS FIRST,i_item_desc#32 ASC NULLS FIRST,s_store_id#29 ASC NULLS FIRST,s_store_name#28 ASC NULLS FIRST], output=[i_item_id#33,i_item_desc#32,s_store_id#29,s_store_name#28,store_sales_profit#37,store_returns_loss#38,catalog_sales_profit#39]), [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39], 100, 0, [i_item_id#33 ASC NULLS FIRST, i_item_desc#32 ASC NULLS FIRST, s_store_id#29 ASC NULLS FIRST, s_store_name#28 ASC NULLS FIRST], [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] + +(44) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_id#33, i_item_desc#32, s_store_id#29, s_store_name#28, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt index cded28ce81..4e1a4f3ec0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, s_store_id#3, s_store_name#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS store_sales_profit#7, MakeDecimal(sum(UnscaledValue(sr_net_loss#8))#9,17,2) AS store_returns_loss#10, MakeDecimal(sum(UnscaledValue(cs_net_profit#11))#12,17,2) AS catalog_sales_profit#13)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -59,4 +59,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 50 out of 57 eligible operators (87%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 52 out of 57 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt index 2140d72398..dcad304452 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] + CometHashAggregate [sum,sum,sum] [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit))] CometExchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 CometHashAggregate [ss_net_profit,sr_net_loss,cs_net_profit] [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] CometProject [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt index 59dc807199..2d4f57a9f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt index 58ab22b65c..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2 AS brand_id#3, i_brand#4 AS brand#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS sum_agg#8)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt index 84c696334f..7bdad4b94f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt index 59dc807199..2d4f57a9f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,sum_agg#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,sum_agg#15]), [d_year#2, brand_id#13, brand#14, sum_agg#15], 100, 0, [d_year#2 ASC NULLS FIRST, sum_agg#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, sum_agg#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#13, brand#14, sum_agg#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt index 58ab22b65c..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2 AS brand_id#3, i_brand#4 AS brand#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS sum_agg#8)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt index 84c696334f..7bdad4b94f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,sum_agg] + CometHashAggregate [sum] [d_year,brand_id,brand,sum_agg,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt index c97d45cc8e..dac625d856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/explain.txt @@ -1,58 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (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.customer_address (9) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometExchange (29) - : : +- CometHashAggregate (28) - : : +- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (20) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (47) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (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_returns (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.customer_address (9) + : : +- CometBroadcastExchange (34) + : : +- CometFilter (33) + : : +- CometHashAggregate (32) + : : +- CometExchange (31) + : : +- CometHashAggregate (30) + : : +- CometHashAggregate (29) + : : +- CometExchange (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (40) + : +- CometProject (39) + : +- CometFilter (38) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns @@ -132,214 +128,191 @@ Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 5] -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] - -(18) HashAggregate [codegen id : 5] +(17) CometHashAggregate Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] Keys [2]: [wr_returning_customer_sk#1, ca_state#10] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#10 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(19) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) +(18) CometFilter +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(wr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [wr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#21] -Arguments: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18], [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] - -(25) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#22, ca_state#23] - -(26) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] -Right output [2]: [ca_address_sk#22, ca_state#23] -Arguments: [wr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight - -(27) CometProject -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#22, ca_state#23] -Arguments: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23], [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] - -(28) CometHashAggregate -Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] -Keys [2]: [wr_returning_customer_sk#16, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] - -(29) CometExchange -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] -Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] - -(31) HashAggregate [codegen id : 1] -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] -Keys [2]: [wr_returning_customer_sk#16, ca_state#23] -Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] -Results [2]: [ca_state#23 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 1] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(34) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +(20) CometFilter +Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Condition : isnotnull(wr_returning_addr_sk#16) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] + +(22) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight + +(23) CometProject +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] +Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] + +(24) ReusedExchange [Reuses operator id: 12] +Output [2]: [ca_address_sk#21, ca_state#22] + +(25) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] +Right output [2]: [ca_address_sk#21, ca_state#22] +Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight + +(26) CometProject +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] +Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] + +(27) CometHashAggregate +Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] + +(28) CometExchange +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] +Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] + +(30) CometHashAggregate +Input [2]: [ctr_state#24, ctr_total_return#25] +Keys [1]: [ctr_state#24] +Functions [1]: [partial_avg(ctr_total_return#25)] + +(31) CometExchange +Input [3]: [ctr_state#24, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(32) CometHashAggregate +Input [3]: [ctr_state#24, sum#26, count#27] +Keys [1]: [ctr_state#24] +Functions [1]: [avg(ctr_total_return#25)] + +(33) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) + +(34) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] + +(35) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight + +(36) CometProject +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(38) CometFilter +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) -(42) CometProject -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Arguments: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#47, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#39, 1)) AS c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#44, 13)) AS c_login#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#45, 50)) AS c_email_address#53, c_last_review_date#46] +(39) CometProject +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#43, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#44, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#46, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#40, 13)) AS c_login#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#41, 50)) AS c_email_address#49, c_last_review_date#42] -(43) CometColumnarToRow [codegen id : 3] -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] +(40) CometBroadcastExchange +Input [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -(44) BroadcastExchange -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Right output [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 5] -Output [14]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] +(42) CometProject +Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#54, ca_state#55] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#50, ca_state#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [ca_address_sk#54, ca_state#55] -Condition : ((isnotnull(ca_state#55) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#55, 2)) = GA)) AND isnotnull(ca_address_sk#54)) +(44) CometFilter +Input [2]: [ca_address_sk#50, ca_state#51] +Condition : ((isnotnull(ca_state#51) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#51, 2)) = GA)) AND isnotnull(ca_address_sk#50)) -(49) CometProject -Input [2]: [ca_address_sk#54, ca_state#55] -Arguments: [ca_address_sk#54], [ca_address_sk#54] +(45) CometProject +Input [2]: [ca_address_sk#50, ca_state#51] +Arguments: [ca_address_sk#50], [ca_address_sk#50] -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#54] +(46) CometBroadcastExchange +Input [1]: [ca_address_sk#50] +Arguments: [ca_address_sk#50] -(51) BroadcastExchange -Input [1]: [ca_address_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(47) CometBroadcastHashJoin +Left output [14]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Right output [1]: [ca_address_sk#50] +Arguments: [c_current_addr_sk#31], [ca_address_sk#50], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#54] -Join type: Inner -Join condition: None +(48) CometProject +Input [15]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ca_address_sk#50] +Arguments: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -(53) Project [codegen id : 5] -Output [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Input [15]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ca_address_sk#54] +(49) CometTakeOrderedAndProject +Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#43 ASC NULLS FIRST,c_salutation#44 ASC NULLS FIRST,c_first_name#45 ASC NULLS FIRST,c_last_name#46 ASC NULLS FIRST,c_preferred_cust_flag#47 ASC NULLS FIRST,c_birth_day#36 ASC NULLS FIRST,c_birth_month#37 ASC NULLS FIRST,c_birth_year#38 ASC NULLS FIRST,c_birth_country#39 ASC NULLS FIRST,c_login#48 ASC NULLS FIRST,c_email_address#49 ASC NULLS FIRST,c_last_review_date#42 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#43,c_salutation#44,c_first_name#45,c_last_name#46,c_preferred_cust_flag#47,c_birth_day#36,c_birth_month#37,c_birth_year#38,c_birth_country#39,c_login#48,c_email_address#49,c_last_review_date#42,ctr_total_return#14]), [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], 100, 0, [c_customer_id#43 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -(54) TakeOrderedAndProject -Input [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Arguments: 100, [c_customer_id#47 ASC NULLS FIRST, c_salutation#48 ASC NULLS FIRST, c_first_name#49 ASC NULLS FIRST, c_last_name#50 ASC NULLS FIRST, c_preferred_cust_flag#51 ASC NULLS FIRST, c_birth_day#40 ASC NULLS FIRST, c_birth_month#41 ASC NULLS FIRST, c_birth_year#42 ASC NULLS FIRST, c_birth_country#43 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#46 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] +(50) CometColumnarToRow [codegen id : 1] +Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(56) CometFilter +(52) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(57) CometProject +(53) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(59) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 20 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt index 38b08219cd..7ebd69d340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/extended.txt @@ -1,69 +1,65 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wr_returning_customer_sk#1 AS ctr_customer_sk#2, ca_state#3 AS ctr_state#4, MakeDecimal(sum(UnscaledValue(wr_return_amt#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- 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.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_state#8 AS ctr_state#9, MakeDecimal(sum(UnscaledValue(wr_return_amt#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 42 out of 61 eligible operators (68%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt index 112d86a057..0227534fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30.native_iceberg_compat/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (5) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #1 CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] @@ -34,43 +34,28 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre 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 #5 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #6 - WholeStageCodegen (1) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - 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] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] + CometExchange [ctr_state] #6 + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] + CometExchange [wr_returning_customer_sk,ca_state] #7 + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] #8 + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometBroadcastExchange [ca_address_sk] #9 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt index c97d45cc8e..dac625d856 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt @@ -1,58 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (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.customer_address (9) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometExchange (29) - : : +- CometHashAggregate (28) - : : +- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (20) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (47) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (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_returns (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.customer_address (9) + : : +- CometBroadcastExchange (34) + : : +- CometFilter (33) + : : +- CometHashAggregate (32) + : : +- CometExchange (31) + : : +- CometHashAggregate (30) + : : +- CometHashAggregate (29) + : : +- CometExchange (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (40) + : +- CometProject (39) + : +- CometFilter (38) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns @@ -132,214 +128,191 @@ Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 5] -Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] - -(18) HashAggregate [codegen id : 5] +(17) CometHashAggregate Input [3]: [wr_returning_customer_sk#1, ca_state#10, sum#11] Keys [2]: [wr_returning_customer_sk#1, ca_state#10] Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#12] -Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#10 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#12,17,2) AS ctr_total_return#15] -(19) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) +(18) CometFilter +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#19), dynamicpruningexpression(wr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(wr_returning_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Condition : isnotnull(wr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [4]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [wr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, wr_returned_date_sk#19, d_date_sk#21] -Arguments: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18], [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] - -(25) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#22, ca_state#23] - -(26) CometBroadcastHashJoin -Left output [3]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18] -Right output [2]: [ca_address_sk#22, ca_state#23] -Arguments: [wr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight - -(27) CometProject -Input [5]: [wr_returning_customer_sk#16, wr_returning_addr_sk#17, wr_return_amt#18, ca_address_sk#22, ca_state#23] -Arguments: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23], [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] - -(28) CometHashAggregate -Input [3]: [wr_returning_customer_sk#16, wr_return_amt#18, ca_state#23] -Keys [2]: [wr_returning_customer_sk#16, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#18))] - -(29) CometExchange -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] -Arguments: hashpartitioning(wr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] - -(31) HashAggregate [codegen id : 1] -Input [3]: [wr_returning_customer_sk#16, ca_state#23, sum#24] -Keys [2]: [wr_returning_customer_sk#16, ca_state#23] -Functions [1]: [sum(UnscaledValue(wr_return_amt#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#18))#12] -Results [2]: [ca_state#23 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 1] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(34) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +(20) CometFilter +Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Condition : isnotnull(wr_returning_addr_sk#16) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] + +(22) CometBroadcastHashJoin +Left output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [wr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight + +(23) CometProject +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] +Arguments: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17], [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] + +(24) ReusedExchange [Reuses operator id: 12] +Output [2]: [ca_address_sk#21, ca_state#22] + +(25) CometBroadcastHashJoin +Left output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] +Right output [2]: [ca_address_sk#21, ca_state#22] +Arguments: [wr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight + +(26) CometProject +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] +Arguments: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22], [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] + +(27) CometHashAggregate +Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] + +(28) CometExchange +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] +Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#23] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] + +(30) CometHashAggregate +Input [2]: [ctr_state#24, ctr_total_return#25] +Keys [1]: [ctr_state#24] +Functions [1]: [partial_avg(ctr_total_return#25)] + +(31) CometExchange +Input [3]: [ctr_state#24, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(32) CometHashAggregate +Input [3]: [ctr_state#24, sum#26, count#27] +Keys [1]: [ctr_state#24] +Functions [1]: [avg(ctr_total_return#25)] + +(33) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) + +(34) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] + +(35) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight + +(36) CometProject +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(38) CometFilter +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) -(42) CometProject -Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] -Arguments: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#47, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#39, 1)) AS c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#44, 13)) AS c_login#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#45, 50)) AS c_email_address#53, c_last_review_date#46] +(39) CometProject +Input [14]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34, c_preferred_cust_flag#35, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#40, c_email_address#41, c_last_review_date#42] +Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#43, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#44, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#46, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#35, 1)) AS c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#40, 13)) AS c_login#48, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#41, 50)) AS c_email_address#49, c_last_review_date#42] -(43) CometColumnarToRow [codegen id : 3] -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] +(40) CometBroadcastExchange +Input [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -(44) BroadcastExchange -Input [14]: [c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Right output [14]: [c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 5] -Output [14]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] -Input [16]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46] +(42) CometProject +Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Arguments: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42], [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#54, ca_state#55] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#50, ca_state#51] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] ReadSchema: struct -(48) CometFilter -Input [2]: [ca_address_sk#54, ca_state#55] -Condition : ((isnotnull(ca_state#55) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#55, 2)) = GA)) AND isnotnull(ca_address_sk#54)) +(44) CometFilter +Input [2]: [ca_address_sk#50, ca_state#51] +Condition : ((isnotnull(ca_state#51) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#51, 2)) = GA)) AND isnotnull(ca_address_sk#50)) -(49) CometProject -Input [2]: [ca_address_sk#54, ca_state#55] -Arguments: [ca_address_sk#54], [ca_address_sk#54] +(45) CometProject +Input [2]: [ca_address_sk#50, ca_state#51] +Arguments: [ca_address_sk#50], [ca_address_sk#50] -(50) CometColumnarToRow [codegen id : 4] -Input [1]: [ca_address_sk#54] +(46) CometBroadcastExchange +Input [1]: [ca_address_sk#50] +Arguments: [ca_address_sk#50] -(51) BroadcastExchange -Input [1]: [ca_address_sk#54] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(47) CometBroadcastHashJoin +Left output [14]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42] +Right output [1]: [ca_address_sk#50] +Arguments: [c_current_addr_sk#31], [ca_address_sk#50], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#54] -Join type: Inner -Join condition: None +(48) CometProject +Input [15]: [ctr_total_return#14, c_customer_id#43, c_current_addr_sk#31, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ca_address_sk#50] +Arguments: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -(53) Project [codegen id : 5] -Output [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Input [15]: [ctr_total_return#15, c_customer_id#47, c_current_addr_sk#35, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ca_address_sk#54] +(49) CometTakeOrderedAndProject +Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#43 ASC NULLS FIRST,c_salutation#44 ASC NULLS FIRST,c_first_name#45 ASC NULLS FIRST,c_last_name#46 ASC NULLS FIRST,c_preferred_cust_flag#47 ASC NULLS FIRST,c_birth_day#36 ASC NULLS FIRST,c_birth_month#37 ASC NULLS FIRST,c_birth_year#38 ASC NULLS FIRST,c_birth_country#39 ASC NULLS FIRST,c_login#48 ASC NULLS FIRST,c_email_address#49 ASC NULLS FIRST,c_last_review_date#42 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#43,c_salutation#44,c_first_name#45,c_last_name#46,c_preferred_cust_flag#47,c_birth_day#36,c_birth_month#37,c_birth_year#38,c_birth_country#39,c_login#48,c_email_address#49,c_last_review_date#42,ctr_total_return#14]), [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14], 100, 0, [c_customer_id#43 ASC NULLS FIRST, c_salutation#44 ASC NULLS FIRST, c_first_name#45 ASC NULLS FIRST, c_last_name#46 ASC NULLS FIRST, c_preferred_cust_flag#47 ASC NULLS FIRST, c_birth_day#36 ASC NULLS FIRST, c_birth_month#37 ASC NULLS FIRST, c_birth_year#38 ASC NULLS FIRST, c_birth_country#39 ASC NULLS FIRST, c_login#48 ASC NULLS FIRST, c_email_address#49 ASC NULLS FIRST, c_last_review_date#42 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] -(54) TakeOrderedAndProject -Input [13]: [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] -Arguments: 100, [c_customer_id#47 ASC NULLS FIRST, c_salutation#48 ASC NULLS FIRST, c_first_name#49 ASC NULLS FIRST, c_last_name#50 ASC NULLS FIRST, c_preferred_cust_flag#51 ASC NULLS FIRST, c_birth_day#40 ASC NULLS FIRST, c_birth_month#41 ASC NULLS FIRST, c_birth_year#42 ASC NULLS FIRST, c_birth_country#43 ASC NULLS FIRST, c_login#52 ASC NULLS FIRST, c_email_address#53 ASC NULLS FIRST, c_last_review_date#46 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#47, c_salutation#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#52, c_email_address#53, c_last_review_date#46, ctr_total_return#15] +(50) CometColumnarToRow [codegen id : 1] +Input [13]: [c_customer_id#43, c_salutation#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_day#36, c_birth_month#37, c_birth_year#38, c_birth_country#39, c_login#48, c_email_address#49, c_last_review_date#42, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(56) CometFilter +(52) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2002)) AND isnotnull(d_date_sk#6)) -(57) CometProject +(53) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(59) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 20 Hosting Expression = wr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt index 38b08219cd..7ebd69d340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/extended.txt @@ -1,69 +1,65 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wr_returning_customer_sk#1 AS ctr_customer_sk#2, ca_state#3 AS ctr_state#4, MakeDecimal(sum(UnscaledValue(wr_return_amt#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : : +- 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.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_state#8 AS ctr_state#9, MakeDecimal(sum(UnscaledValue(wr_return_amt#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 42 out of 61 eligible operators (68%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt index 112d86a057..0227534fa5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - WholeStageCodegen (5) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ca_address_sk] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,wr_returning_customer_sk,ca_state,sum(UnscaledValue(wr_return_amt))] CometExchange [wr_returning_customer_sk,ca_state] #1 CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] @@ -34,43 +34,28 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_pre 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 #5 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #6 - WholeStageCodegen (1) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [wr_returning_customer_sk,ca_state] #7 - CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] - CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] - CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] - CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] - CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - 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] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] + CometExchange [ctr_state] #6 + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [wr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(wr_return_amt))] + CometExchange [wr_returning_customer_sk,ca_state] #7 + CometHashAggregate [wr_return_amt] [wr_returning_customer_sk,ca_state,sum] + CometProject [wr_returning_customer_sk,wr_return_amt,ca_state] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,ca_address_sk,ca_state] + CometProject [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + CometBroadcastHashJoin [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk,d_date_sk] + CometFilter [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] #8 + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_login,c_email_address] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + CometBroadcastExchange [ca_address_sk] #9 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt index 97da1fbdfd..83d717e295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/explain.txt @@ -1,100 +1,94 @@ == Physical Plan == -* CometColumnarToRow (96) -+- CometSort (95) - +- CometColumnarExchange (94) - +- * Project (93) - +- * BroadcastHashJoin Inner BuildRight (92) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * BroadcastHashJoin Inner BuildRight (32) - : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- BroadcastExchange (31) - : : : : +- * HashAggregate (30) - : : : : +- * CometColumnarToRow (29) - : : : : +- CometExchange (28) - : : : : +- CometHashAggregate (27) - : : : : +- CometProject (26) - : : : : +- CometBroadcastHashJoin (25) - : : : : :- CometProject (23) - : : : : : +- CometBroadcastHashJoin (22) - : : : : : :- CometFilter (18) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - : : : : : +- CometBroadcastExchange (21) - : : : : : +- CometFilter (20) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : : : +- ReusedExchange (24) - : : : +- BroadcastExchange (47) - : : : +- * HashAggregate (46) - : : : +- * CometColumnarToRow (45) - : : : +- CometExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometProject (42) - : : : +- CometBroadcastHashJoin (41) - : : : :- CometProject (39) - : : : : +- CometBroadcastHashJoin (38) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) - : : : : +- CometBroadcastExchange (37) - : : : : +- CometFilter (36) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - : : : +- ReusedExchange (40) - : : +- BroadcastExchange (62) - : : +- * HashAggregate (61) - : : +- * CometColumnarToRow (60) - : : +- CometExchange (59) - : : +- CometHashAggregate (58) - : : +- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- BroadcastExchange (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - +- BroadcastExchange (91) - +- * HashAggregate (90) - +- * CometColumnarToRow (89) - +- CometExchange (88) - +- CometHashAggregate (87) - +- CometProject (86) - +- CometBroadcastHashJoin (85) - :- CometProject (83) - : +- CometBroadcastHashJoin (82) - : :- CometFilter (80) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : +- ReusedExchange (81) - +- ReusedExchange (84) +* CometColumnarToRow (90) ++- CometSort (89) + +- CometExchange (88) + +- CometProject (87) + +- CometBroadcastHashJoin (86) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (59) + : : :- CometProject (46) + : : : +- CometBroadcastHashJoin (45) + : : : :- CometBroadcastHashJoin (30) + : : : : :- CometHashAggregate (15) + : : : : : +- CometExchange (14) + : : : : : +- CometHashAggregate (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) + : : : : +- CometBroadcastExchange (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometExchange (27) + : : : : +- CometHashAggregate (26) + : : : : +- CometProject (25) + : : : : +- CometBroadcastHashJoin (24) + : : : : :- CometProject (22) + : : : : : +- CometBroadcastHashJoin (21) + : : : : : :- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) + : : : : : +- CometBroadcastExchange (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (23) + : : : +- CometBroadcastExchange (44) + : : : +- CometHashAggregate (43) + : : : +- CometExchange (42) + : : : +- CometHashAggregate (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (32) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : : : +- CometBroadcastExchange (35) + : : : : +- CometFilter (34) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (58) + : : +- CometHashAggregate (57) + : : +- CometExchange (56) + : : +- CometHashAggregate (55) + : : +- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometFilter (48) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (52) + : +- CometBroadcastExchange (71) + : +- CometHashAggregate (70) + : +- CometExchange (69) + : +- CometHashAggregate (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometFilter (61) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- CometBroadcastExchange (85) + +- CometHashAggregate (84) + +- CometExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometFilter (75) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -166,469 +160,434 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(15) CometColumnarToRow [codegen id : 6] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] - -(16) HashAggregate [codegen id : 6] +(15) CometHashAggregate Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12] -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_addr_sk#13) +(17) CometFilter +Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_addr_sk#11) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) - -(21) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [d_date_sk#17, d_year#18, d_qoy#19] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Right output [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19], [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] - -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#20, ca_county#21] - -(25) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] -Right output [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ss_addr_sk#13], [ca_address_sk#20], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] -Arguments: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21], [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] - -(27) CometHashAggregate -Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] - -(28) CometExchange -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] - -(30) HashAggregate [codegen id : 1] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#14))#11] -Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#14))#11,17,2) AS store_sales#23] - -(31) BroadcastExchange -Input [2]: [ca_county#21, store_sales#23] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#21] -Join type: Inner -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(19) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) + +(20) CometBroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [d_date_sk#15, d_year#16, d_qoy#17] + +(21) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight + +(22) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#18, ca_county#19] + +(24) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] +Right output [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight + +(25) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] +Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] + +(26) CometHashAggregate +Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] + +(27) CometExchange +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] + +(29) CometBroadcastExchange +Input [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#19, store_sales#21] + +(30) CometBroadcastHashJoin +Left output [3]: [ca_county#9, d_year#6, store_sales#22] +Right output [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight + +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#23, ss_ext_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_addr_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_addr_sk#24) +(32) CometFilter +Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_addr_sk#23) -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) +(34) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) + +(35) CometBroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [d_date_sk#27, d_year#28, d_qoy#29] + +(36) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(37) CometBroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [d_date_sk#28, d_year#29, d_qoy#30] +(37) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -(38) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight +(38) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#30, ca_county#31] -(39) CometProject -Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30], [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] +(39) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] +Right output [2]: [ca_address_sk#30, ca_county#31] +Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight -(40) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#31, ca_county#32] +(40) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] +Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -(41) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] -Right output [2]: [ca_address_sk#31, ca_county#32] -Arguments: [ss_addr_sk#24], [ca_address_sk#31], Inner, BuildRight +(41) CometHashAggregate +Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] -(42) CometProject -Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_address_sk#31, ca_county#32] -Arguments: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32], [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] +(42) CometExchange +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (43) CometHashAggregate -Input [4]: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] -Keys [3]: [ca_county#32, d_qoy#30, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#25))] - -(44) CometExchange -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] -Arguments: hashpartitioning(ca_county#32, d_qoy#30, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 2] -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] - -(46) HashAggregate [codegen id : 2] -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] -Keys [3]: [ca_county#32, d_qoy#30, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#25))#11] -Results [2]: [ca_county#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#25))#11,17,2) AS store_sales#34] - -(47) BroadcastExchange -Input [2]: [ca_county#32, store_sales#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(48) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#21] -Right keys [1]: [ca_county#32] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 6] -Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34] -Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#23, ca_county#32, store_sales#34] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] + +(44) CometBroadcastExchange +Input [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#31, store_sales#33] + +(45) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] +Right output [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight + +(46) CometProject +Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] + +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] -Condition : isnotnull(ws_bill_addr_sk#35) +(48) CometFilter +Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_addr_sk#34) + +(49) ReusedExchange [Reuses operator id: 5] +Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +(51) CometProject +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] + +(52) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#41, ca_county#42] (53) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] -Right output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight +Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] +Right output [2]: [ca_address_sk#41, ca_county#42] +Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight (54) CometProject -Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] -Arguments: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41], [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] - -(55) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#42, ca_county#43] - -(56) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] -Right output [2]: [ca_address_sk#42, ca_county#43] -Arguments: [ws_bill_addr_sk#35], [ca_address_sk#42], Inner, BuildRight - -(57) CometProject -Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#42, ca_county#43] -Arguments: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43], [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] - -(58) CometHashAggregate -Input [4]: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] -Keys [3]: [ca_county#43, d_qoy#41, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))] - -(59) CometExchange -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] -Arguments: hashpartitioning(ca_county#43, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(60) CometColumnarToRow [codegen id : 3] -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] - -(61) HashAggregate [codegen id : 3] -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] -Keys [3]: [ca_county#43, d_qoy#41, d_year#40] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#45] -Results [2]: [ca_county#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#45,17,2) AS web_sales#46] - -(62) BroadcastExchange -Input [2]: [ca_county#43, web_sales#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(63) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#43] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] +Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] + +(55) CometHashAggregate +Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] + +(56) CometExchange +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(57) CometHashAggregate +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] + +(58) CometBroadcastExchange +Input [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#42, web_sales#44] + +(59) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] +Right output [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight + +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(65) CometFilter -Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] -Condition : isnotnull(ws_bill_addr_sk#47) - -(66) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] - -(67) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] -Right output [3]: [d_date_sk#51, d_year#52, d_qoy#53] -Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight - -(68) CometProject -Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52, d_qoy#53] -Arguments: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53], [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#54, ca_county#55] - -(70) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] -Right output [2]: [ca_address_sk#54, ca_county#55] -Arguments: [ws_bill_addr_sk#47], [ca_address_sk#54], Inner, BuildRight - -(71) CometProject -Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_address_sk#54, ca_county#55] -Arguments: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55], [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] - -(72) CometHashAggregate -Input [4]: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] -Keys [3]: [ca_county#55, d_qoy#53, d_year#52] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))] - -(73) CometExchange -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] -Arguments: hashpartitioning(ca_county#55, d_qoy#53, d_year#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(74) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] - -(75) HashAggregate [codegen id : 4] -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] -Keys [3]: [ca_county#55, d_qoy#53, d_year#52] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#45] -Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#45,17,2) AS web_sales#57] - -(76) BroadcastExchange -Input [2]: [ca_county#55, web_sales#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] - -(77) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#43] -Right keys [1]: [ca_county#55] -Join type: Inner -Join condition: (CASE WHEN (web_sales#46 > 0.00) THEN (web_sales#57 / web_sales#46) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#23 / store_sales#12) END) - -(78) Project [codegen id : 6] -Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57] -Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, ca_county#55, web_sales#57] - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] +(61) CometFilter +Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_addr_sk#45) + +(62) ReusedExchange [Reuses operator id: 20] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] + +(63) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight + +(64) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] + +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#52, ca_county#53] + +(66) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] +Right output [2]: [ca_address_sk#52, ca_county#53] +Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight + +(67) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] +Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] + +(68) CometHashAggregate +Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] + +(69) CometExchange +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(70) CometHashAggregate +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] + +(71) CometBroadcastExchange +Input [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#53, web_sales#55] + +(72) CometBroadcastHashJoin +Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] +Right output [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight + +(73) CometProject +Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] + +(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) CometFilter -Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_addr_sk#58) +(75) CometFilter +Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_addr_sk#56) -(81) ReusedExchange [Reuses operator id: 37] -Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] +(76) ReusedExchange [Reuses operator id: 35] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -(82) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] -Right output [3]: [d_date_sk#62, d_year#63, d_qoy#64] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight +(77) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight -(83) CometProject -Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63, d_qoy#64] -Arguments: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64], [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] +(78) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -(84) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#65, ca_county#66] +(79) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#63, ca_county#64] -(85) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] -Right output [2]: [ca_address_sk#65, ca_county#66] -Arguments: [ws_bill_addr_sk#58], [ca_address_sk#65], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] +Right output [2]: [ca_address_sk#63, ca_county#64] +Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight -(86) CometProject -Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_address_sk#65, ca_county#66] -Arguments: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66], [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] +(81) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] +Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -(87) CometHashAggregate -Input [4]: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] -Keys [3]: [ca_county#66, d_qoy#64, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))] +(82) CometHashAggregate +Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] -(88) CometExchange -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] -Arguments: hashpartitioning(ca_county#66, d_qoy#64, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(83) CometExchange +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(89) CometColumnarToRow [codegen id : 5] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +(84) CometHashAggregate +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] -(90) HashAggregate [codegen id : 5] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] -Keys [3]: [ca_county#66, d_qoy#64, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#45] -Results [2]: [ca_county#66, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#45,17,2) AS web_sales#68] +(85) CometBroadcastExchange +Input [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#64, web_sales#66] -(91) BroadcastExchange -Input [2]: [ca_county#66, web_sales#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(86) CometBroadcastHashJoin +Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] +Right output [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight -(92) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#43] -Right keys [1]: [ca_county#66] -Join type: Inner -Join condition: (CASE WHEN (web_sales#57 > 0.00) THEN (web_sales#68 / web_sales#57) END > CASE WHEN (store_sales#23 > 0.00) THEN (store_sales#34 / store_sales#23) END) +(87) CometProject +Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(93) Project [codegen id : 6] -Output [6]: [ca_county#9, d_year#6, (web_sales#57 / web_sales#46) AS web_q1_q2_increase#69, (store_sales#23 / store_sales#12) AS store_q1_q2_increase#70, (web_sales#68 / web_sales#57) AS web_q2_q3_increase#71, (store_sales#34 / store_sales#23) AS store_q2_q3_increase#72] -Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57, ca_county#66, web_sales#68] - -(94) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(88) CometExchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(95) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72], [ca_county#9 ASC NULLS FIRST] +(89) CometSort +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(96) CometColumnarToRow [codegen id : 7] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] +(90) CometColumnarToRow [codegen id : 1] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometFilter (98) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (97) +BroadcastExchange (94) ++- * CometColumnarToRow (93) + +- CometFilter (92) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(98) CometFilter +(92) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(99) CometColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(100) BroadcastExchange +(94) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometFilter (102) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (101) +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (98) ++- * CometColumnarToRow (97) + +- CometFilter (96) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) -(101) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(102) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) +(96) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(103) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(97) CometColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -(104) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] +(98) BroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometFilter (106) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (105) +Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (102) ++- * CometColumnarToRow (101) + +- CometFilter (100) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) -(105) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(106) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) +(100) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(107) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(101) CometColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -(108) BroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +(102) BroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt index aef97a2068..b4e7aa4db6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/extended.txt @@ -1,132 +1,126 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#1, d_year#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS store_sales#5)] - : : : : : +- CometColumnarToRow - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- 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.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#4,17,2) AS store_sales#8)] - : : : : +- CometColumnarToRow - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- 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.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#10))#4,17,2) AS store_sales#11)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.customer_address - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#13))#14,17,2) AS web_sales#15)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#16, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#17))#14,17,2) AS web_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#20))#14,17,2) AS web_sales#21)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- 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.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- 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.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 92 out of 120 eligible operators (76%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt index fcea2cb245..2f97384320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31.native_iceberg_compat/simplified.txt @@ -1,139 +1,116 @@ -WholeStageCodegen (7) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 - WholeStageCodegen (6) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #4 - 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 [ca_address_sk,ca_county] #5 - 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 (1) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #9 + CometExchange [ca_county] #1 + CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] + CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #2 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter 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] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #13 - 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] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (4) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #18 - WholeStageCodegen (5) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + 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 [ca_address_sk,ca_county] #5 + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [ca_county,store_sales] #6 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #7 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + 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,d_year,d_qoy] #9 + 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] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,store_sales] #10 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #11 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #12 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + 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,d_year,d_qoy] #13 + 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] ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #14 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #15 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #16 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #17 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #18 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #19 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt index 97da1fbdfd..83d717e295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt @@ -1,100 +1,94 @@ == Physical Plan == -* CometColumnarToRow (96) -+- CometSort (95) - +- CometColumnarExchange (94) - +- * Project (93) - +- * BroadcastHashJoin Inner BuildRight (92) - :- * Project (78) - : +- * BroadcastHashJoin Inner BuildRight (77) - : :- * BroadcastHashJoin Inner BuildRight (63) - : : :- * Project (49) - : : : +- * BroadcastHashJoin Inner BuildRight (48) - : : : :- * BroadcastHashJoin Inner BuildRight (32) - : : : : :- * HashAggregate (16) - : : : : : +- * CometColumnarToRow (15) - : : : : : +- CometExchange (14) - : : : : : +- CometHashAggregate (13) - : : : : : +- CometProject (12) - : : : : : +- CometBroadcastHashJoin (11) - : : : : : :- CometProject (7) - : : : : : : +- CometBroadcastHashJoin (6) - : : : : : : :- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometBroadcastExchange (5) - : : : : : : +- CometFilter (4) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : : : : +- CometBroadcastExchange (10) - : : : : : +- CometFilter (9) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) - : : : : +- BroadcastExchange (31) - : : : : +- * HashAggregate (30) - : : : : +- * CometColumnarToRow (29) - : : : : +- CometExchange (28) - : : : : +- CometHashAggregate (27) - : : : : +- CometProject (26) - : : : : +- CometBroadcastHashJoin (25) - : : : : :- CometProject (23) - : : : : : +- CometBroadcastHashJoin (22) - : : : : : :- CometFilter (18) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (17) - : : : : : +- CometBroadcastExchange (21) - : : : : : +- CometFilter (20) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (19) - : : : : +- ReusedExchange (24) - : : : +- BroadcastExchange (47) - : : : +- * HashAggregate (46) - : : : +- * CometColumnarToRow (45) - : : : +- CometExchange (44) - : : : +- CometHashAggregate (43) - : : : +- CometProject (42) - : : : +- CometBroadcastHashJoin (41) - : : : :- CometProject (39) - : : : : +- CometBroadcastHashJoin (38) - : : : : :- CometFilter (34) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) - : : : : +- CometBroadcastExchange (37) - : : : : +- CometFilter (36) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (35) - : : : +- ReusedExchange (40) - : : +- BroadcastExchange (62) - : : +- * HashAggregate (61) - : : +- * CometColumnarToRow (60) - : : +- CometExchange (59) - : : +- CometHashAggregate (58) - : : +- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometProject (54) - : : : +- CometBroadcastHashJoin (53) - : : : :- CometFilter (51) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (50) - : : : +- ReusedExchange (52) - : : +- ReusedExchange (55) - : +- BroadcastExchange (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- CometExchange (73) - : +- CometHashAggregate (72) - : +- CometProject (71) - : +- CometBroadcastHashJoin (70) - : :- CometProject (68) - : : +- CometBroadcastHashJoin (67) - : : :- CometFilter (65) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (64) - : : +- ReusedExchange (66) - : +- ReusedExchange (69) - +- BroadcastExchange (91) - +- * HashAggregate (90) - +- * CometColumnarToRow (89) - +- CometExchange (88) - +- CometHashAggregate (87) - +- CometProject (86) - +- CometBroadcastHashJoin (85) - :- CometProject (83) - : +- CometBroadcastHashJoin (82) - : :- CometFilter (80) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (79) - : +- ReusedExchange (81) - +- ReusedExchange (84) +* CometColumnarToRow (90) ++- CometSort (89) + +- CometExchange (88) + +- CometProject (87) + +- CometBroadcastHashJoin (86) + :- CometProject (73) + : +- CometBroadcastHashJoin (72) + : :- CometBroadcastHashJoin (59) + : : :- CometProject (46) + : : : +- CometBroadcastHashJoin (45) + : : : :- CometBroadcastHashJoin (30) + : : : : :- CometHashAggregate (15) + : : : : : +- CometExchange (14) + : : : : : +- CometHashAggregate (13) + : : : : : +- CometProject (12) + : : : : : +- CometBroadcastHashJoin (11) + : : : : : :- CometProject (7) + : : : : : : +- CometBroadcastHashJoin (6) + : : : : : : :- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometBroadcastExchange (5) + : : : : : : +- CometFilter (4) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : : : : +- CometBroadcastExchange (10) + : : : : : +- CometFilter (9) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (8) + : : : : +- CometBroadcastExchange (29) + : : : : +- CometHashAggregate (28) + : : : : +- CometExchange (27) + : : : : +- CometHashAggregate (26) + : : : : +- CometProject (25) + : : : : +- CometBroadcastHashJoin (24) + : : : : :- CometProject (22) + : : : : : +- CometBroadcastHashJoin (21) + : : : : : :- CometFilter (17) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (16) + : : : : : +- CometBroadcastExchange (20) + : : : : : +- CometFilter (19) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) + : : : : +- ReusedExchange (23) + : : : +- CometBroadcastExchange (44) + : : : +- CometHashAggregate (43) + : : : +- CometExchange (42) + : : : +- CometHashAggregate (41) + : : : +- CometProject (40) + : : : +- CometBroadcastHashJoin (39) + : : : :- CometProject (37) + : : : : +- CometBroadcastHashJoin (36) + : : : : :- CometFilter (32) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : : : +- CometBroadcastExchange (35) + : : : : +- CometFilter (34) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) + : : : +- ReusedExchange (38) + : : +- CometBroadcastExchange (58) + : : +- CometHashAggregate (57) + : : +- CometExchange (56) + : : +- CometHashAggregate (55) + : : +- CometProject (54) + : : +- CometBroadcastHashJoin (53) + : : :- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometFilter (48) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : : +- ReusedExchange (49) + : : +- ReusedExchange (52) + : +- CometBroadcastExchange (71) + : +- CometHashAggregate (70) + : +- CometExchange (69) + : +- CometHashAggregate (68) + : +- CometProject (67) + : +- CometBroadcastHashJoin (66) + : :- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometFilter (61) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- CometBroadcastExchange (85) + +- CometHashAggregate (84) + +- CometExchange (83) + +- CometHashAggregate (82) + +- CometProject (81) + +- CometBroadcastHashJoin (80) + :- CometProject (78) + : +- CometBroadcastHashJoin (77) + : :- CometFilter (75) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (76) + +- ReusedExchange (79) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -166,469 +160,434 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(15) CometColumnarToRow [codegen id : 6] -Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] - -(16) HashAggregate [codegen id : 6] +(15) CometHashAggregate Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#10] Keys [3]: [ca_county#9, d_qoy#7, d_year#6] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#11] -Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS store_sales#12] -(17) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] +(16) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] PushedFilters: [IsNotNull(ss_addr_sk)] ReadSchema: struct -(18) CometFilter -Input [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_addr_sk#13) +(17) CometFilter +Input [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Condition : isnotnull(ss_addr_sk#11) -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(20) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) - -(21) CometBroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [d_date_sk#17, d_year#18, d_qoy#19] - -(22) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15] -Right output [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight - -(23) CometProject -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_year#18, d_qoy#19] -Arguments: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19], [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] - -(24) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#20, ca_county#21] - -(25) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19] -Right output [2]: [ca_address_sk#20, ca_county#21] -Arguments: [ss_addr_sk#13], [ca_address_sk#20], Inner, BuildRight - -(26) CometProject -Input [6]: [ss_addr_sk#13, ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_address_sk#20, ca_county#21] -Arguments: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21], [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] - -(27) CometHashAggregate -Input [4]: [ss_ext_sales_price#14, d_year#18, d_qoy#19, ca_county#21] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#14))] - -(28) CometExchange -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -Arguments: hashpartitioning(ca_county#21, d_qoy#19, d_year#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(29) CometColumnarToRow [codegen id : 1] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] - -(30) HashAggregate [codegen id : 1] -Input [4]: [ca_county#21, d_qoy#19, d_year#18, sum#22] -Keys [3]: [ca_county#21, d_qoy#19, d_year#18] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#14))#11] -Results [2]: [ca_county#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#14))#11,17,2) AS store_sales#23] - -(31) BroadcastExchange -Input [2]: [ca_county#21, store_sales#23] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(32) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#21] -Join type: Inner -Join condition: None - -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +(19) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) + +(20) CometBroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [d_date_sk#15, d_year#16, d_qoy#17] + +(21) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13] +Right output [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_sold_date_sk#13], [d_date_sk#15], Inner, BuildRight + +(22) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_year#16, d_qoy#17] +Arguments: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17], [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#18, ca_county#19] + +(24) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17] +Right output [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ss_addr_sk#11], [ca_address_sk#18], Inner, BuildRight + +(25) CometProject +Input [6]: [ss_addr_sk#11, ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_address_sk#18, ca_county#19] +Arguments: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19], [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] + +(26) CometHashAggregate +Input [4]: [ss_ext_sales_price#12, d_year#16, d_qoy#17, ca_county#19] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#12))] + +(27) CometExchange +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Arguments: hashpartitioning(ca_county#19, d_qoy#17, d_year#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [4]: [ca_county#19, d_qoy#17, d_year#16, sum#20] +Keys [3]: [ca_county#19, d_qoy#17, d_year#16] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#12))] + +(29) CometBroadcastExchange +Input [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#19, store_sales#21] + +(30) CometBroadcastHashJoin +Left output [3]: [ca_county#9, d_year#6, store_sales#22] +Right output [2]: [ca_county#19, store_sales#21] +Arguments: [ca_county#9], [ca_county#19], Inner, BuildRight + +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#23, ss_ext_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_addr_sk)] ReadSchema: struct -(34) CometFilter -Input [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_addr_sk#24) +(32) CometFilter +Input [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_addr_sk#23) -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(36) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) +(34) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) + +(35) CometBroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [d_date_sk#27, d_year#28, d_qoy#29] + +(36) CometBroadcastHashJoin +Left output [3]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25] +Right output [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight -(37) CometBroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [d_date_sk#28, d_year#29, d_qoy#30] +(37) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_year#28, d_qoy#29] +Arguments: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29], [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] -(38) CometBroadcastHashJoin -Left output [3]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] -Right output [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight +(38) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#30, ca_county#31] -(39) CometProject -Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29, d_qoy#30] -Arguments: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30], [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] +(39) CometBroadcastHashJoin +Left output [4]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29] +Right output [2]: [ca_address_sk#30, ca_county#31] +Arguments: [ss_addr_sk#23], [ca_address_sk#30], Inner, BuildRight -(40) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#31, ca_county#32] +(40) CometProject +Input [6]: [ss_addr_sk#23, ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_address_sk#30, ca_county#31] +Arguments: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31], [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] -(41) CometBroadcastHashJoin -Left output [4]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30] -Right output [2]: [ca_address_sk#31, ca_county#32] -Arguments: [ss_addr_sk#24], [ca_address_sk#31], Inner, BuildRight +(41) CometHashAggregate +Input [4]: [ss_ext_sales_price#24, d_year#28, d_qoy#29, ca_county#31] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#24))] -(42) CometProject -Input [6]: [ss_addr_sk#24, ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_address_sk#31, ca_county#32] -Arguments: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32], [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] +(42) CometExchange +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Arguments: hashpartitioning(ca_county#31, d_qoy#29, d_year#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (43) CometHashAggregate -Input [4]: [ss_ext_sales_price#25, d_year#29, d_qoy#30, ca_county#32] -Keys [3]: [ca_county#32, d_qoy#30, d_year#29] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#25))] - -(44) CometExchange -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] -Arguments: hashpartitioning(ca_county#32, d_qoy#30, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(45) CometColumnarToRow [codegen id : 2] -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] - -(46) HashAggregate [codegen id : 2] -Input [4]: [ca_county#32, d_qoy#30, d_year#29, sum#33] -Keys [3]: [ca_county#32, d_qoy#30, d_year#29] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#25))#11] -Results [2]: [ca_county#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#25))#11,17,2) AS store_sales#34] - -(47) BroadcastExchange -Input [2]: [ca_county#32, store_sales#34] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(48) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#21] -Right keys [1]: [ca_county#32] -Join type: Inner -Join condition: None - -(49) Project [codegen id : 6] -Output [5]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34] -Input [7]: [ca_county#9, d_year#6, store_sales#12, ca_county#21, store_sales#23, ca_county#32, store_sales#34] - -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] +Input [4]: [ca_county#31, d_qoy#29, d_year#28, sum#32] +Keys [3]: [ca_county#31, d_qoy#29, d_year#28] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#24))] + +(44) CometBroadcastExchange +Input [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#31, store_sales#33] + +(45) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21] +Right output [2]: [ca_county#31, store_sales#33] +Arguments: [ca_county#19], [ca_county#31], Inner, BuildRight + +(46) CometProject +Input [7]: [ca_county#9, d_year#6, store_sales#22, ca_county#19, store_sales#21, ca_county#31, store_sales#33] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] + +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#37), dynamicpruningexpression(ws_sold_date_sk#37 IN dynamicpruning#38)] +PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(51) CometFilter -Input [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] -Condition : isnotnull(ws_bill_addr_sk#35) +(48) CometFilter +Input [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Condition : isnotnull(ws_bill_addr_sk#34) + +(49) ReusedExchange [Reuses operator id: 5] +Output [3]: [d_date_sk#38, d_year#39, d_qoy#40] + +(50) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +Right output [3]: [d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 5] -Output [3]: [d_date_sk#39, d_year#40, d_qoy#41] +(51) CometProject +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38, d_year#39, d_qoy#40] +Arguments: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40], [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] + +(52) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#41, ca_county#42] (53) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37] -Right output [3]: [d_date_sk#39, d_year#40, d_qoy#41] -Arguments: [ws_sold_date_sk#37], [d_date_sk#39], Inner, BuildRight +Left output [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40] +Right output [2]: [ca_address_sk#41, ca_county#42] +Arguments: [ws_bill_addr_sk#34], [ca_address_sk#41], Inner, BuildRight (54) CometProject -Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, ws_sold_date_sk#37, d_date_sk#39, d_year#40, d_qoy#41] -Arguments: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41], [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] - -(55) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#42, ca_county#43] - -(56) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41] -Right output [2]: [ca_address_sk#42, ca_county#43] -Arguments: [ws_bill_addr_sk#35], [ca_address_sk#42], Inner, BuildRight - -(57) CometProject -Input [6]: [ws_bill_addr_sk#35, ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_address_sk#42, ca_county#43] -Arguments: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43], [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] - -(58) CometHashAggregate -Input [4]: [ws_ext_sales_price#36, d_year#40, d_qoy#41, ca_county#43] -Keys [3]: [ca_county#43, d_qoy#41, d_year#40] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#36))] - -(59) CometExchange -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] -Arguments: hashpartitioning(ca_county#43, d_qoy#41, d_year#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(60) CometColumnarToRow [codegen id : 3] -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] - -(61) HashAggregate [codegen id : 3] -Input [4]: [ca_county#43, d_qoy#41, d_year#40, sum#44] -Keys [3]: [ca_county#43, d_qoy#41, d_year#40] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#36))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#36))#45] -Results [2]: [ca_county#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#36))#45,17,2) AS web_sales#46] - -(62) BroadcastExchange -Input [2]: [ca_county#43, web_sales#46] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(63) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#9] -Right keys [1]: [ca_county#43] -Join type: Inner -Join condition: None - -(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] +Input [6]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_address_sk#41, ca_county#42] +Arguments: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42], [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] + +(55) CometHashAggregate +Input [4]: [ws_ext_sales_price#35, d_year#39, d_qoy#40, ca_county#42] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] + +(56) CometExchange +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Arguments: hashpartitioning(ca_county#42, d_qoy#40, d_year#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(57) CometHashAggregate +Input [4]: [ca_county#42, d_qoy#40, d_year#39, sum#43] +Keys [3]: [ca_county#42, d_qoy#40, d_year#39] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] + +(58) CometBroadcastExchange +Input [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#42, web_sales#44] + +(59) CometBroadcastHashJoin +Left output [5]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33] +Right output [2]: [ca_county#42, web_sales#44] +Arguments: [ca_county#9], [ca_county#42], Inner, BuildRight + +(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#49), dynamicpruningexpression(ws_sold_date_sk#49 IN dynamicpruning#50)] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(65) CometFilter -Input [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] -Condition : isnotnull(ws_bill_addr_sk#47) - -(66) ReusedExchange [Reuses operator id: 21] -Output [3]: [d_date_sk#51, d_year#52, d_qoy#53] - -(67) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49] -Right output [3]: [d_date_sk#51, d_year#52, d_qoy#53] -Arguments: [ws_sold_date_sk#49], [d_date_sk#51], Inner, BuildRight - -(68) CometProject -Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, ws_sold_date_sk#49, d_date_sk#51, d_year#52, d_qoy#53] -Arguments: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53], [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] - -(69) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#54, ca_county#55] - -(70) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53] -Right output [2]: [ca_address_sk#54, ca_county#55] -Arguments: [ws_bill_addr_sk#47], [ca_address_sk#54], Inner, BuildRight - -(71) CometProject -Input [6]: [ws_bill_addr_sk#47, ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_address_sk#54, ca_county#55] -Arguments: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55], [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] - -(72) CometHashAggregate -Input [4]: [ws_ext_sales_price#48, d_year#52, d_qoy#53, ca_county#55] -Keys [3]: [ca_county#55, d_qoy#53, d_year#52] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#48))] - -(73) CometExchange -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] -Arguments: hashpartitioning(ca_county#55, d_qoy#53, d_year#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(74) CometColumnarToRow [codegen id : 4] -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] - -(75) HashAggregate [codegen id : 4] -Input [4]: [ca_county#55, d_qoy#53, d_year#52, sum#56] -Keys [3]: [ca_county#55, d_qoy#53, d_year#52] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#48))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#48))#45] -Results [2]: [ca_county#55, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#48))#45,17,2) AS web_sales#57] - -(76) BroadcastExchange -Input [2]: [ca_county#55, web_sales#57] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=9] - -(77) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#43] -Right keys [1]: [ca_county#55] -Join type: Inner -Join condition: (CASE WHEN (web_sales#46 > 0.00) THEN (web_sales#57 / web_sales#46) END > CASE WHEN (store_sales#12 > 0.00) THEN (store_sales#23 / store_sales#12) END) - -(78) Project [codegen id : 6] -Output [8]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57] -Input [9]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, ca_county#55, web_sales#57] - -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] +(61) CometFilter +Input [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_addr_sk#45) + +(62) ReusedExchange [Reuses operator id: 20] +Output [3]: [d_date_sk#49, d_year#50, d_qoy#51] + +(63) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47] +Right output [3]: [d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_sold_date_sk#47], [d_date_sk#49], Inner, BuildRight + +(64) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, ws_sold_date_sk#47, d_date_sk#49, d_year#50, d_qoy#51] +Arguments: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51], [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] + +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#52, ca_county#53] + +(66) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51] +Right output [2]: [ca_address_sk#52, ca_county#53] +Arguments: [ws_bill_addr_sk#45], [ca_address_sk#52], Inner, BuildRight + +(67) CometProject +Input [6]: [ws_bill_addr_sk#45, ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_address_sk#52, ca_county#53] +Arguments: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53], [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] + +(68) CometHashAggregate +Input [4]: [ws_ext_sales_price#46, d_year#50, d_qoy#51, ca_county#53] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#46))] + +(69) CometExchange +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Arguments: hashpartitioning(ca_county#53, d_qoy#51, d_year#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(70) CometHashAggregate +Input [4]: [ca_county#53, d_qoy#51, d_year#50, sum#54] +Keys [3]: [ca_county#53, d_qoy#51, d_year#50] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#46))] + +(71) CometBroadcastExchange +Input [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#53, web_sales#55] + +(72) CometBroadcastHashJoin +Left output [7]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44] +Right output [2]: [ca_county#53, web_sales#55] +Arguments: [ca_county#42], [ca_county#53], Inner, (CASE WHEN (web_sales#44 > 0.00) THEN (web_sales#55 / web_sales#44) END > CASE WHEN (store_sales#22 > 0.00) THEN (store_sales#21 / store_sales#22) END), BuildRight + +(73) CometProject +Input [9]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, ca_county#53, web_sales#55] +Arguments: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55], [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] + +(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_bill_addr_sk)] ReadSchema: struct -(80) CometFilter -Input [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] -Condition : isnotnull(ws_bill_addr_sk#58) +(75) CometFilter +Input [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Condition : isnotnull(ws_bill_addr_sk#56) -(81) ReusedExchange [Reuses operator id: 37] -Output [3]: [d_date_sk#62, d_year#63, d_qoy#64] +(76) ReusedExchange [Reuses operator id: 35] +Output [3]: [d_date_sk#60, d_year#61, d_qoy#62] -(82) CometBroadcastHashJoin -Left output [3]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60] -Right output [3]: [d_date_sk#62, d_year#63, d_qoy#64] -Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight +(77) CometBroadcastHashJoin +Left output [3]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58] +Right output [3]: [d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_sold_date_sk#58], [d_date_sk#60], Inner, BuildRight -(83) CometProject -Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63, d_qoy#64] -Arguments: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64], [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] +(78) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, ws_sold_date_sk#58, d_date_sk#60, d_year#61, d_qoy#62] +Arguments: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62], [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] -(84) ReusedExchange [Reuses operator id: 10] -Output [2]: [ca_address_sk#65, ca_county#66] +(79) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#63, ca_county#64] -(85) CometBroadcastHashJoin -Left output [4]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64] -Right output [2]: [ca_address_sk#65, ca_county#66] -Arguments: [ws_bill_addr_sk#58], [ca_address_sk#65], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [4]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62] +Right output [2]: [ca_address_sk#63, ca_county#64] +Arguments: [ws_bill_addr_sk#56], [ca_address_sk#63], Inner, BuildRight -(86) CometProject -Input [6]: [ws_bill_addr_sk#58, ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_address_sk#65, ca_county#66] -Arguments: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66], [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] +(81) CometProject +Input [6]: [ws_bill_addr_sk#56, ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_address_sk#63, ca_county#64] +Arguments: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64], [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] -(87) CometHashAggregate -Input [4]: [ws_ext_sales_price#59, d_year#63, d_qoy#64, ca_county#66] -Keys [3]: [ca_county#66, d_qoy#64, d_year#63] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#59))] +(82) CometHashAggregate +Input [4]: [ws_ext_sales_price#57, d_year#61, d_qoy#62, ca_county#64] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#57))] -(88) CometExchange -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] -Arguments: hashpartitioning(ca_county#66, d_qoy#64, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(83) CometExchange +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Arguments: hashpartitioning(ca_county#64, d_qoy#62, d_year#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(89) CometColumnarToRow [codegen id : 5] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] +(84) CometHashAggregate +Input [4]: [ca_county#64, d_qoy#62, d_year#61, sum#65] +Keys [3]: [ca_county#64, d_qoy#62, d_year#61] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#57))] -(90) HashAggregate [codegen id : 5] -Input [4]: [ca_county#66, d_qoy#64, d_year#63, sum#67] -Keys [3]: [ca_county#66, d_qoy#64, d_year#63] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#59))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#59))#45] -Results [2]: [ca_county#66, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#45,17,2) AS web_sales#68] +(85) CometBroadcastExchange +Input [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#64, web_sales#66] -(91) BroadcastExchange -Input [2]: [ca_county#66, web_sales#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] +(86) CometBroadcastHashJoin +Left output [8]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55] +Right output [2]: [ca_county#64, web_sales#66] +Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.00) THEN (web_sales#66 / web_sales#55) END > CASE WHEN (store_sales#21 > 0.00) THEN (store_sales#33 / store_sales#21) END), BuildRight -(92) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ca_county#43] -Right keys [1]: [ca_county#66] -Join type: Inner -Join condition: (CASE WHEN (web_sales#57 > 0.00) THEN (web_sales#68 / web_sales#57) END > CASE WHEN (store_sales#23 > 0.00) THEN (store_sales#34 / store_sales#23) END) +(87) CometProject +Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(93) Project [codegen id : 6] -Output [6]: [ca_county#9, d_year#6, (web_sales#57 / web_sales#46) AS web_q1_q2_increase#69, (store_sales#23 / store_sales#12) AS store_q1_q2_increase#70, (web_sales#68 / web_sales#57) AS web_q2_q3_increase#71, (store_sales#34 / store_sales#23) AS store_q2_q3_increase#72] -Input [10]: [ca_county#9, d_year#6, store_sales#12, store_sales#23, store_sales#34, ca_county#43, web_sales#46, web_sales#57, ca_county#66, web_sales#68] - -(94) CometColumnarExchange -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(88) CometExchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(95) CometSort -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] -Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72], [ca_county#9 ASC NULLS FIRST] +(89) CometSort +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] +Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9 ASC NULLS FIRST] -(96) CometColumnarToRow [codegen id : 7] -Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#69, store_q1_q2_increase#70, web_q2_q3_increase#71, store_q2_q3_increase#72] +(90) CometColumnarToRow [codegen id : 1] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (100) -+- * CometColumnarToRow (99) - +- CometFilter (98) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (97) +BroadcastExchange (94) ++- * CometColumnarToRow (93) + +- CometFilter (92) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (91) -(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(91) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(98) CometFilter +(92) CometFilter Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) -(99) CometColumnarToRow [codegen id : 1] +(93) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -(100) BroadcastExchange +(94) BroadcastExchange Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 17 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#16 -BroadcastExchange (104) -+- * CometColumnarToRow (103) - +- CometFilter (102) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (101) +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (98) ++- * CometColumnarToRow (97) + +- CometFilter (96) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (95) -(101) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(95) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_year#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(102) CometFilter -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Condition : ((((isnotnull(d_qoy#19) AND isnotnull(d_year#18)) AND (d_qoy#19 = 2)) AND (d_year#18 = 2000)) AND isnotnull(d_date_sk#17)) +(96) CometFilter +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Condition : ((((isnotnull(d_qoy#17) AND isnotnull(d_year#16)) AND (d_qoy#17 = 2)) AND (d_year#16 = 2000)) AND isnotnull(d_date_sk#15)) -(103) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] +(97) CometColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] -(104) BroadcastExchange -Input [3]: [d_date_sk#17, d_year#18, d_qoy#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] +(98) BroadcastExchange +Input [3]: [d_date_sk#15, d_year#16, d_qoy#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:3 Hosting operator id = 33 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 -BroadcastExchange (108) -+- * CometColumnarToRow (107) - +- CometFilter (106) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (105) +Subquery:3 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#26 +BroadcastExchange (102) ++- * CometColumnarToRow (101) + +- CometFilter (100) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (99) -(105) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(99) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#27, d_year#28, d_qoy#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] ReadSchema: struct -(106) CometFilter -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Condition : ((((isnotnull(d_qoy#30) AND isnotnull(d_year#29)) AND (d_qoy#30 = 3)) AND (d_year#29 = 2000)) AND isnotnull(d_date_sk#28)) +(100) CometFilter +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Condition : ((((isnotnull(d_qoy#29) AND isnotnull(d_year#28)) AND (d_qoy#29 = 3)) AND (d_year#28 = 2000)) AND isnotnull(d_date_sk#27)) -(107) CometColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] +(101) CometColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] -(108) BroadcastExchange -Input [3]: [d_date_sk#28, d_year#29, d_qoy#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] +(102) BroadcastExchange +Input [3]: [d_date_sk#27, d_year#28, d_qoy#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] -Subquery:4 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#37 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 47 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 64 Hosting Expression = ws_sold_date_sk#49 IN dynamicpruning#16 +Subquery:5 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#14 -Subquery:6 Hosting operator id = 79 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#26 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt index aef97a2068..b4e7aa4db6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/extended.txt @@ -1,132 +1,126 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Project - : : : +- BroadcastHashJoin - : : : :- BroadcastHashJoin - : : : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#1, d_year#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS store_sales#5)] - : : : : : +- CometColumnarToRow - : : : : : +- CometExchange - : : : : : +- CometHashAggregate - : : : : : +- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- 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.customer_address - : : : : +- BroadcastExchange - : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#4,17,2) AS store_sales#8)] - : : : : +- CometColumnarToRow - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- 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.customer_address - : : : +- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#10))#4,17,2) AS store_sales#11)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- 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.customer_address - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#13))#14,17,2) AS web_sales#15)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#16, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#17))#14,17,2) AS web_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_county#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#20))#14,17,2) AS web_sales#21)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometBroadcastHashJoin + : : : : :- CometHashAggregate + : : : : : +- CometExchange + : : : : : +- CometHashAggregate + : : : : : +- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : : +- SubqueryBroadcast + : : : : : : : +- BroadcastExchange + : : : : : : : +- 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.customer_address + : : : : +- CometBroadcastExchange + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- 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.customer_address + : : : +- CometBroadcastExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 92 out of 120 eligible operators (76%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 111 out of 120 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt index fcea2cb245..2f97384320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -1,139 +1,116 @@ -WholeStageCodegen (7) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 - WholeStageCodegen (6) - Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] - BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - Project [ca_county,d_year,store_sales,store_sales,store_sales] - BroadcastHashJoin [ca_county,ca_county] - BroadcastHashJoin [ca_county,ca_county] - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #2 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #4 - 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 [ca_address_sk,ca_county] #5 - 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 (1) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #7 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #8 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #9 + CometExchange [ca_county] #1 + CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,ca_county,web_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales] + CometProject [ca_county,d_year,store_sales,store_sales,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales,ca_county,store_sales] + CometBroadcastHashJoin [ca_county,d_year,store_sales,ca_county,store_sales] + CometHashAggregate [d_qoy,sum] [ca_county,d_year,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #2 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter 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] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #11 - CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #3 - BroadcastExchange #12 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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,d_year,d_qoy] #13 - 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] - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (3) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #15 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_year,d_qoy] #4 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #16 - WholeStageCodegen (4) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #17 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #2 - ReusedExchange [d_date_sk,d_year,d_qoy] #9 - ReusedExchange [ca_address_sk,ca_county] #5 - InputAdapter - BroadcastExchange #18 - WholeStageCodegen (5) - HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [ca_county,d_qoy,d_year] #19 - CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] - CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] - CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] - CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] - CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #3 - ReusedExchange [d_date_sk,d_year,d_qoy] #13 + CometBroadcastExchange [d_date_sk,d_year,d_qoy] #4 + 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 [ca_address_sk,ca_county] #5 + CometFilter [ca_address_sk,ca_county] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + CometBroadcastExchange [ca_county,store_sales] #6 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #7 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + 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,d_year,d_qoy] #9 + 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] + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,store_sales] #10 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,store_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #11 + CometHashAggregate [ss_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ss_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #12 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + 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,d_year,d_qoy] #13 + 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] ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #14 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #15 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_year,d_qoy] #4 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #16 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #17 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + ReusedExchange [d_date_sk,d_year,d_qoy] #9 + ReusedExchange [ca_address_sk,ca_county] #5 + CometBroadcastExchange [ca_county,web_sales] #18 + CometHashAggregate [d_qoy,d_year,sum] [ca_county,web_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [ca_county,d_qoy,d_year] #19 + CometHashAggregate [ws_ext_sales_price] [ca_county,d_qoy,d_year,sum] + CometProject [ws_ext_sales_price,d_year,d_qoy,ca_county] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy,ca_address_sk,ca_county] + CometProject [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + CometBroadcastHashJoin [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk,d_year,d_qoy] + CometFilter [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + ReusedExchange [d_date_sk,d_year,d_qoy] #13 + ReusedExchange [ca_address_sk,ca_county] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt index 0117667c41..d6c10008ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) +* CometColumnarToRow (30) ++- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) +- CometProject (26) @@ -163,15 +163,13 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] Input [1]: [sum#17] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#17] - -(30) HashAggregate [codegen id : 1] +(29) CometHashAggregate Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#18] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#18,17,2) AS excess discount amount#19] + +(30) CometColumnarToRow [codegen id : 1] +Input [1]: [excess discount amount#18] ===== Subqueries ===== @@ -184,18 +182,18 @@ BroadcastExchange (35) (31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#20] +Output [2]: [d_date_sk#16, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#16, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 2000-01-27)) AND (d_date#20 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) (33) CometProject -Input [2]: [d_date_sk#16, d_date#20] +Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] (34) CometColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt index 86e50e4751..943bb147ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#1))#2,17,2) AS excess discount amount#3)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 38 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt index e095592cc2..07619fc999 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 CometHashAggregate [cs_ext_discount_amt] [sum] CometProject [cs_ext_discount_amt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt index 0117667c41..d6c10008ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) +* CometColumnarToRow (30) ++- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) +- CometProject (26) @@ -163,15 +163,13 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] Input [1]: [sum#17] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#17] - -(30) HashAggregate [codegen id : 1] +(29) CometHashAggregate Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#18] -Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#18,17,2) AS excess discount amount#19] + +(30) CometColumnarToRow [codegen id : 1] +Input [1]: [excess discount amount#18] ===== Subqueries ===== @@ -184,18 +182,18 @@ BroadcastExchange (35) (31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#20] +Output [2]: [d_date_sk#16, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#16, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 2000-01-27)) AND (d_date#20 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) (33) CometProject -Input [2]: [d_date_sk#16, d_date#20] +Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] (34) CometColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt index 86e50e4751..943bb147ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#1))#2,17,2) AS excess discount amount#3)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 38 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt index e095592cc2..07619fc999 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [excess discount amount,sum(UnscaledValue(cs_ext_discount_amt))] CometExchange #1 CometHashAggregate [cs_ext_discount_amt] [sum] CometProject [cs_ext_discount_amt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt index 87a45ed1d1..b7f7807f53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/explain.txt @@ -1,68 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (64) -+- * HashAggregate (63) - +- * CometColumnarToRow (62) - +- CometColumnarExchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- * HashAggregate (58) - +- * CometColumnarToRow (57) - +- CometExchange (56) - +- CometHashAggregate (55) - +- CometProject (54) - +- CometBroadcastHashJoin (53) - :- CometProject (51) - : +- CometBroadcastHashJoin (50) - : :- CometProject (48) - : : +- CometBroadcastHashJoin (47) - : : :- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) +* CometColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -190,211 +187,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_manufact_id#12, sum#15] Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, sum#15] - -(28) HashAggregate [codegen id : 1] +(27) CometHashAggregate Input [2]: [i_manufact_id#12, sum#15] Keys [1]: [i_manufact_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [2]: [i_manufact_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_manufact_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_manufact_id#24] -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_manufact_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_manufact_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_manufact_id#26] -Arguments: [cs_ext_sales_price#20, i_manufact_id#26], [cs_ext_sales_price#20, i_manufact_id#26] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] +Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_manufact_id#26] -Keys [1]: [i_manufact_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] +Keys [1]: [i_manufact_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) CometExchange -Input [2]: [i_manufact_id#26, sum#27] -Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [2]: [i_manufact_id#24, sum#25] +Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [2]: [i_manufact_id#26, sum#27] +(41) CometHashAggregate +Input [2]: [i_manufact_id#24, sum#25] +Keys [1]: [i_manufact_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(43) HashAggregate [codegen id : 2] -Input [2]: [i_manufact_id#26, sum#27] -Keys [1]: [i_manufact_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] -Results [2]: [i_manufact_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) - -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(48) CometProject -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_manufact_id#38] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_manufact_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_manufact_id#34] -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] -Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38] +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_manufact_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(55) CometHashAggregate -Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Keys [1]: [i_manufact_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] +Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] -(56) CometExchange -Input [2]: [i_manufact_id#38, sum#39] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Keys [1]: [i_manufact_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(57) CometColumnarToRow [codegen id : 3] -Input [2]: [i_manufact_id#38, sum#39] +(54) CometExchange +Input [2]: [i_manufact_id#34, sum#35] +Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(58) HashAggregate [codegen id : 3] -Input [2]: [i_manufact_id#38, sum#39] -Keys [1]: [i_manufact_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_manufact_id#34, sum#35] +Keys [1]: [i_manufact_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_manufact_id#12, total_sales#36] +Child 1 Input [2]: [i_manufact_id#24, total_sales#37] +Child 2 Input [2]: [i_manufact_id#34, total_sales#38] -(60) HashAggregate [codegen id : 4] -Input [2]: [i_manufact_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_manufact_id#12, total_sales#36] Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) CometColumnarExchange -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(58) CometExchange +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(62) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] - -(63) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_manufact_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(64) TakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47] +(61) CometColumnarToRow [codegen id : 1] +Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (66) ++- * CometColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter +(63) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) -(67) CometProject +(64) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(68) CometColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(69) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt index 0085008392..063fdb6faf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/extended.txt @@ -1,100 +1,97 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS total_sales#4)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS total_sales#8)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#9, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#10))#11,17,2) AS total_sales#12)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 82 out of 93 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt index 23325abbf1..b4b84c2ac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33.native_iceberg_compat/simplified.txt @@ -1,84 +1,73 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (5) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (2) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - WholeStageCodegen (3) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] + CometExchange [i_manufact_id] #1 + CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] + CometUnion [i_manufact_id,total_sales] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_manufact_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #7 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_manufact_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_manufact_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt index 87a45ed1d1..b7f7807f53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt @@ -1,68 +1,65 @@ == Physical Plan == -TakeOrderedAndProject (64) -+- * HashAggregate (63) - +- * CometColumnarToRow (62) - +- CometColumnarExchange (61) - +- * HashAggregate (60) - +- Union (59) - :- * HashAggregate (28) - : +- * CometColumnarToRow (27) - : +- CometExchange (26) - : +- CometHashAggregate (25) - : +- CometProject (24) - : +- CometBroadcastHashJoin (23) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometFilter (30) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) - : : : +- ReusedExchange (31) - : : +- ReusedExchange (34) - : +- ReusedExchange (37) - +- * HashAggregate (58) - +- * CometColumnarToRow (57) - +- CometExchange (56) - +- CometHashAggregate (55) - +- CometProject (54) - +- CometBroadcastHashJoin (53) - :- CometProject (51) - : +- CometBroadcastHashJoin (50) - : :- CometProject (48) - : : +- CometBroadcastHashJoin (47) - : : :- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- ReusedExchange (46) - : +- ReusedExchange (49) - +- ReusedExchange (52) +* CometColumnarToRow (61) ++- CometTakeOrderedAndProject (60) + +- CometHashAggregate (59) + +- CometExchange (58) + +- CometHashAggregate (57) + +- CometUnion (56) + :- CometHashAggregate (27) + : +- CometExchange (26) + : +- CometHashAggregate (25) + : +- CometProject (24) + : +- CometBroadcastHashJoin (23) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometFilter (29) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (28) + : : : +- ReusedExchange (30) + : : +- ReusedExchange (33) + : +- ReusedExchange (36) + +- CometHashAggregate (55) + +- CometExchange (54) + +- CometHashAggregate (53) + +- CometProject (52) + +- CometBroadcastHashJoin (51) + :- CometProject (49) + : +- CometBroadcastHashJoin (48) + : :- CometProject (46) + : : +- CometBroadcastHashJoin (45) + : : :- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- ReusedExchange (44) + : +- ReusedExchange (47) + +- ReusedExchange (50) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -190,211 +187,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_manufact_id#12, sum#15] Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(27) CometColumnarToRow [codegen id : 1] -Input [2]: [i_manufact_id#12, sum#15] - -(28) HashAggregate [codegen id : 1] +(27) CometHashAggregate Input [2]: [i_manufact_id#12, sum#15] Keys [1]: [i_manufact_id#12] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [2]: [i_manufact_id#12, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS total_sales#17] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] +PartitionFilters: [isnotnull(cs_sold_date_sk#19), dynamicpruningexpression(cs_sold_date_sk#19 IN dynamicpruning#20)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(30) CometFilter -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) +(29) CometFilter +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Condition : (isnotnull(cs_bill_addr_sk#16) AND isnotnull(cs_item_sk#17)) -(31) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#23] +(30) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#21] -(32) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] -Right output [1]: [d_date_sk#23] -Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19] +Right output [1]: [d_date_sk#21] +Arguments: [cs_sold_date_sk#19], [d_date_sk#21], Inner, BuildRight -(33) CometProject -Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] -Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +(32) CometProject +Input [5]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, cs_sold_date_sk#19, d_date_sk#21] +Arguments: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18], [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] -(34) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#24] +(33) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#22] -(35) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -Right output [1]: [ca_address_sk#24] -Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight +(34) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18] +Right output [1]: [ca_address_sk#22] +Arguments: [cs_bill_addr_sk#16], [ca_address_sk#22], Inner, BuildRight -(36) CometProject -Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] -Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] +(35) CometProject +Input [4]: [cs_bill_addr_sk#16, cs_item_sk#17, cs_ext_sales_price#18, ca_address_sk#22] +Arguments: [cs_item_sk#17, cs_ext_sales_price#18], [cs_item_sk#17, cs_ext_sales_price#18] -(37) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#25, i_manufact_id#26] +(36) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#23, i_manufact_id#24] -(38) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] -Right output [2]: [i_item_sk#25, i_manufact_id#26] -Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#17, cs_ext_sales_price#18] +Right output [2]: [i_item_sk#23, i_manufact_id#24] +Arguments: [cs_item_sk#17], [i_item_sk#23], Inner, BuildRight -(39) CometProject -Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_manufact_id#26] -Arguments: [cs_ext_sales_price#20, i_manufact_id#26], [cs_ext_sales_price#20, i_manufact_id#26] +(38) CometProject +Input [4]: [cs_item_sk#17, cs_ext_sales_price#18, i_item_sk#23, i_manufact_id#24] +Arguments: [cs_ext_sales_price#18, i_manufact_id#24], [cs_ext_sales_price#18, i_manufact_id#24] -(40) CometHashAggregate -Input [2]: [cs_ext_sales_price#20, i_manufact_id#26] -Keys [1]: [i_manufact_id#26] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] +(39) CometHashAggregate +Input [2]: [cs_ext_sales_price#18, i_manufact_id#24] +Keys [1]: [i_manufact_id#24] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#18))] -(41) CometExchange -Input [2]: [i_manufact_id#26, sum#27] -Arguments: hashpartitioning(i_manufact_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [2]: [i_manufact_id#24, sum#25] +Arguments: hashpartitioning(i_manufact_id#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [2]: [i_manufact_id#26, sum#27] +(41) CometHashAggregate +Input [2]: [i_manufact_id#24, sum#25] +Keys [1]: [i_manufact_id#24] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#18))] -(43) HashAggregate [codegen id : 2] -Input [2]: [i_manufact_id#26, sum#27] -Keys [1]: [i_manufact_id#26] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#20))#28] -Results [2]: [i_manufact_id#26, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#20))#28,17,2) AS total_sales#29] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] 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#29), dynamicpruningexpression(ws_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) - -(46) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#35] +(43) CometFilter +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Condition : (isnotnull(ws_bill_addr_sk#27) AND isnotnull(ws_item_sk#26)) -(47) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] -Right output [1]: [d_date_sk#35] -Arguments: [ws_sold_date_sk#33], [d_date_sk#35], Inner, BuildRight +(44) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#31] -(48) CometProject -Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] -Arguments: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32], [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +(45) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29] +Right output [1]: [d_date_sk#31] +Arguments: [ws_sold_date_sk#29], [d_date_sk#31], Inner, BuildRight -(49) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#36] +(46) CometProject +Input [5]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ws_sold_date_sk#29, d_date_sk#31] +Arguments: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28], [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] -(50) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] -Right output [1]: [ca_address_sk#36] -Arguments: [ws_bill_addr_sk#31], [ca_address_sk#36], Inner, BuildRight +(47) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#32] -(51) CometProject -Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] -Arguments: [ws_item_sk#30, ws_ext_sales_price#32], [ws_item_sk#30, ws_ext_sales_price#32] +(48) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28] +Right output [1]: [ca_address_sk#32] +Arguments: [ws_bill_addr_sk#27], [ca_address_sk#32], Inner, BuildRight -(52) ReusedExchange [Reuses operator id: 22] -Output [2]: [i_item_sk#37, i_manufact_id#38] +(49) CometProject +Input [4]: [ws_item_sk#26, ws_bill_addr_sk#27, ws_ext_sales_price#28, ca_address_sk#32] +Arguments: [ws_item_sk#26, ws_ext_sales_price#28], [ws_item_sk#26, ws_ext_sales_price#28] -(53) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#30, ws_ext_sales_price#32] -Right output [2]: [i_item_sk#37, i_manufact_id#38] -Arguments: [ws_item_sk#30], [i_item_sk#37], Inner, BuildRight +(50) ReusedExchange [Reuses operator id: 22] +Output [2]: [i_item_sk#33, i_manufact_id#34] -(54) CometProject -Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] -Arguments: [ws_ext_sales_price#32, i_manufact_id#38], [ws_ext_sales_price#32, i_manufact_id#38] +(51) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#26, ws_ext_sales_price#28] +Right output [2]: [i_item_sk#33, i_manufact_id#34] +Arguments: [ws_item_sk#26], [i_item_sk#33], Inner, BuildRight -(55) CometHashAggregate -Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] -Keys [1]: [i_manufact_id#38] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +(52) CometProject +Input [4]: [ws_item_sk#26, ws_ext_sales_price#28, i_item_sk#33, i_manufact_id#34] +Arguments: [ws_ext_sales_price#28, i_manufact_id#34], [ws_ext_sales_price#28, i_manufact_id#34] -(56) CometExchange -Input [2]: [i_manufact_id#38, sum#39] -Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(53) CometHashAggregate +Input [2]: [ws_ext_sales_price#28, i_manufact_id#34] +Keys [1]: [i_manufact_id#34] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#28))] -(57) CometColumnarToRow [codegen id : 3] -Input [2]: [i_manufact_id#38, sum#39] +(54) CometExchange +Input [2]: [i_manufact_id#34, sum#35] +Arguments: hashpartitioning(i_manufact_id#34, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(58) HashAggregate [codegen id : 3] -Input [2]: [i_manufact_id#38, sum#39] -Keys [1]: [i_manufact_id#38] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#40] -Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#40,17,2) AS total_sales#41] +(55) CometHashAggregate +Input [2]: [i_manufact_id#34, sum#35] +Keys [1]: [i_manufact_id#34] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#28))] -(59) Union +(56) CometUnion +Child 0 Input [2]: [i_manufact_id#12, total_sales#36] +Child 1 Input [2]: [i_manufact_id#24, total_sales#37] +Child 2 Input [2]: [i_manufact_id#34, total_sales#38] -(60) HashAggregate [codegen id : 4] -Input [2]: [i_manufact_id#12, total_sales#17] +(57) CometHashAggregate +Input [2]: [i_manufact_id#12, total_sales#36] Keys [1]: [i_manufact_id#12] -Functions [1]: [partial_sum(total_sales#17)] -Aggregate Attributes [2]: [sum#42, isEmpty#43] -Results [3]: [i_manufact_id#12, sum#44, isEmpty#45] +Functions [1]: [partial_sum(total_sales#36)] -(61) CometColumnarExchange -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] -Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(58) CometExchange +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(62) CometColumnarToRow [codegen id : 5] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] - -(63) HashAggregate [codegen id : 5] -Input [3]: [i_manufact_id#12, sum#44, isEmpty#45] +(59) CometHashAggregate +Input [3]: [i_manufact_id#12, sum#39, isEmpty#40] Keys [1]: [i_manufact_id#12] -Functions [1]: [sum(total_sales#17)] -Aggregate Attributes [1]: [sum(total_sales#17)#46] -Results [2]: [i_manufact_id#12, sum(total_sales#17)#46 AS total_sales#47] +Functions [1]: [sum(total_sales#36)] + +(60) CometTakeOrderedAndProject +Input [2]: [i_manufact_id#12, total_sales#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#41 ASC NULLS FIRST], output=[i_manufact_id#12,total_sales#41]), [i_manufact_id#12, total_sales#41], 100, 0, [total_sales#41 ASC NULLS FIRST], [i_manufact_id#12, total_sales#41] -(64) TakeOrderedAndProject -Input [2]: [i_manufact_id#12, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#12, total_sales#47] +(61) CometColumnarToRow [codegen id : 1] +Input [2]: [i_manufact_id#12, total_sales#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (66) ++- * CometColumnarToRow (65) + +- CometProject (64) + +- CometFilter (63) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (62) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(62) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter +(63) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) -(67) CometProject +(64) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(68) CometColumnarToRow [codegen id : 1] +(65) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(69) BroadcastExchange +(66) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 28 Hosting Expression = cs_sold_date_sk#19 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#29 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt index 0085008392..063fdb6faf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/extended.txt @@ -1,100 +1,97 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS total_sales#4)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS total_sales#8)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#9, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#10))#11,17,2) AS total_sales#12)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 82 out of 93 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 89 out of 93 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt index 23325abbf1..b4b84c2ac6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -1,84 +1,73 @@ -TakeOrderedAndProject [total_sales,i_manufact_id] - WholeStageCodegen (5) - HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (4) - HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] - CometProject [ss_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_manufact_id] #6 - CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] - CometFilter [i_item_sk,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] - CometBroadcastExchange [i_manufact_id] #7 - CometProject [i_manufact_id] - CometFilter [i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] - WholeStageCodegen (2) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] - CometProject [cs_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 - WholeStageCodegen (3) - HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] - CometProject [ws_ext_sales_price,i_manufact_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_manufact_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_manufact_id,total_sales,sum(total_sales)] + CometExchange [i_manufact_id] #1 + CometHashAggregate [total_sales] [i_manufact_id,sum,isEmpty] + CometUnion [i_manufact_id,total_sales] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_manufact_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_manufact_id,sum] + CometProject [ss_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_manufact_id] #6 + CometBroadcastHashJoin [i_item_sk,i_manufact_id,i_manufact_id] + CometFilter [i_item_sk,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + CometBroadcastExchange [i_manufact_id] #7 + CometProject [i_manufact_id] + CometFilter [i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_category,i_manufact_id] + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_manufact_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_manufact_id,sum] + CometProject [cs_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 + CometHashAggregate [sum] [i_manufact_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_manufact_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_manufact_id,sum] + CometProject [ws_ext_sales_price,i_manufact_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_manufact_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_manufact_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt index 3c6285353c..634c01506e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt @@ -1,35 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (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) +TakeOrderedAndProject (30) ++- * Project (29) + +- Window (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -141,68 +140,63 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometColumnarToRow [codegen id : 1] -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] - -(25) HashAggregate [codegen id : 1] +(24) CometHashAggregate Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#21, sum(UnscaledValue(ss_ext_sales_price#3))#22] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS gross_margin#23, i_category#16, i_class#17, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS lochierarchy#24, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS _w0#25, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS _w1#26, CASE WHEN (cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint) = 0) THEN i_category#16 END AS _w2#27] -(26) CometColumnarExchange -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(25) CometExchange +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(27) CometSort -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] +(26) CometSort +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 2] -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +(27) CometColumnarToRow [codegen id : 1] +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -(29) Window -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] +(28) Window +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(30) Project [codegen id : 3] -Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +(29) Project [codegen id : 2] +Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -(31) TakeOrderedAndProject -Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(30) TakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * CometColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(31) 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 -(33) CometFilter +(32) 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)) -(34) CometProject +(33) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(35) CometColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#7] 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-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt index c59aac4501..89c523a388 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt @@ -3,37 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt index 6e4a6a969b..61782e3011 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt @@ -1,45 +1,42 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - 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 #3 - 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] #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 [i_item_sk,i_class,i_category] #5 - 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] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + 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 #3 + 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] #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 [i_item_sk,i_class,i_category] #5 + 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] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt index 3c6285353c..634c01506e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -1,35 +1,34 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometColumnarExchange (26) - +- * HashAggregate (25) - +- * CometColumnarToRow (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometExpand (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) +TakeOrderedAndProject (30) ++- * Project (29) + +- Window (28) + +- * CometColumnarToRow (27) + +- CometSort (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -141,68 +140,63 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] Arguments: hashpartitioning(i_category#16, i_class#17, spark_grouping_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometColumnarToRow [codegen id : 1] -Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] - -(25) HashAggregate [codegen id : 1] +(24) CometHashAggregate Input [5]: [i_category#16, i_class#17, spark_grouping_id#18, sum#19, sum#20] Keys [3]: [i_category#16, i_class#17, spark_grouping_id#18] Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#21, sum(UnscaledValue(ss_ext_sales_price#3))#22] -Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS gross_margin#23, i_category#16, i_class#17, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS lochierarchy#24, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#21,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2)) AS _w0#25, (cast((shiftright(spark_grouping_id#18, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint)) AS _w1#26, CASE WHEN (cast((shiftright(spark_grouping_id#18, 0) & 1) as tinyint) = 0) THEN i_category#16 END AS _w2#27] -(26) CometColumnarExchange -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(25) CometExchange +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(27) CometSort -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] +(26) CometSort +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 2] -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +(27) CometColumnarToRow [codegen id : 1] +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -(29) Window -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] +(28) Window +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(30) Project [codegen id : 3] -Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +(29) Project [codegen id : 2] +Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -(31) TakeOrderedAndProject -Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(30) TakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (36) -+- * CometColumnarToRow (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) +BroadcastExchange (35) ++- * CometColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(31) 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 -(33) CometFilter +(32) 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)) -(34) CometProject +(33) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(35) CometColumnarToRow [codegen id : 1] +(34) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(36) BroadcastExchange +(35) BroadcastExchange Input [1]: [d_date_sk#7] 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-v1_4-spark4_0/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt index c59aac4501..89c523a388 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt @@ -3,37 +3,36 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 6e4a6a969b..61782e3011 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -1,45 +1,42 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - 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 #3 - 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] #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 [i_item_sk,i_class,i_category] #5 - 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] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + 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 #3 + 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] #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 [i_item_sk,i_class,i_category] #5 + 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] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt index 2302caa369..add87bddfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#11] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_category_id#8, i_category#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#14] -Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#14] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt index e456cafdfc..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_category_id#2, i_category#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sum(ss_ext_sales_price)#6)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt index 037bb43470..839937d40c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (1) - HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] + CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_category_id,i_category] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt index 2302caa369..add87bddfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_category_id#8, i_category#11, sum#12] Keys [3]: [d_year#2, i_category_id#8, i_category#11] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_category_id#8, i_category#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#14] -Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#14] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sum(ss_ext_sales_price)#13 DESC NULLS LAST,d_year#2 ASC NULLS FIRST,i_category_id#8 ASC NULLS FIRST,i_category#11 ASC NULLS FIRST], output=[d_year#2,i_category_id#8,i_category#11,sum(ss_ext_sales_price)#13]), [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13], 100, 0, [sum(ss_ext_sales_price)#13 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, i_category_id#8, i_category#11, sum(ss_ext_sales_price)#13] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt index e456cafdfc..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_category_id#2, i_category#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sum(ss_ext_sales_price)#6)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt index 037bb43470..839937d40c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] - WholeStageCodegen (1) - HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,i_category_id,i_category,sum(ss_ext_sales_price)] + CometHashAggregate [sum] [d_year,i_category_id,i_category,sum(ss_ext_sales_price),sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_category_id,i_category] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_category_id,i_category,sum] CometProject [d_year,ss_ext_sales_price,i_category_id,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt index d946a0b4c7..ae3885cc8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) T Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [2]: [s_store_name#10, s_store_id#12] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))#26] -Results [9]: [s_store_name#10, s_store_id#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END))#20,17,2) AS sun_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END))#21,17,2) AS mon_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END))#22,17,2) AS tue_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END))#23,17,2) AS wed_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END))#24,17,2) AS thu_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END))#25,17,2) AS fri_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))#26,17,2) AS sat_sales#33] -(19) TakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#27 ASC NULLS FIRST, mon_sales#28 ASC NULLS FIRST, tue_sales#29 ASC NULLS FIRST, wed_sales#30 ASC NULLS FIRST, thu_sales#31 ASC NULLS FIRST, fri_sales#32 ASC NULLS FIRST, sat_sales#33 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33] +(18) CometTakeOrderedAndProject +Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] + +(19) CometColumnarToRow [codegen id : 1] +Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt index 61dd9837e5..cb125e2456 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_name#1, s_store_id#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#4 END))#5,17,2) AS sun_sales#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#4 END))#7,17,2) AS mon_sales#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#4 END))#9,17,2) AS tue_sales#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#4 END))#11,17,2) AS wed_sales#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#4 END))#13,17,2) AS thu_sales#14, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#4 END))#15,17,2) AS fri_sales#16, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#4 END))#17,17,2) AS sat_sales#18)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt index ae850c9d3c..59ad7611a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (1) - HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] CometExchange [s_store_name,s_store_id] #1 CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt index d946a0b4c7..ae3885cc8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) T Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Arguments: hashpartitioning(s_store_name#10, s_store_id#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [9]: [s_store_name#10, s_store_id#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19] Keys [2]: [s_store_name#10, s_store_id#12] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END)), sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))#26] -Results [9]: [s_store_name#10, s_store_id#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Sunday ) THEN ss_sales_price#6 END))#20,17,2) AS sun_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Monday ) THEN ss_sales_price#6 END))#21,17,2) AS mon_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Tuesday ) THEN ss_sales_price#6 END))#22,17,2) AS tue_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Wednesday) THEN ss_sales_price#6 END))#23,17,2) AS wed_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Thursday ) THEN ss_sales_price#6 END))#24,17,2) AS thu_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Friday ) THEN ss_sales_price#6 END))#25,17,2) AS fri_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#4 = Saturday ) THEN ss_sales_price#6 END))#26,17,2) AS sat_sales#33] -(19) TakeOrderedAndProject -Input [9]: [s_store_name#10, s_store_id#12, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33] -Arguments: 100, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#27 ASC NULLS FIRST, mon_sales#28 ASC NULLS FIRST, tue_sales#29 ASC NULLS FIRST, wed_sales#30 ASC NULLS FIRST, thu_sales#31 ASC NULLS FIRST, fri_sales#32 ASC NULLS FIRST, sat_sales#33 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#27, mon_sales#28, tue_sales#29, wed_sales#30, thu_sales#31, fri_sales#32, sat_sales#33] +(18) CometTakeOrderedAndProject +Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#10 ASC NULLS FIRST,s_store_id#12 ASC NULLS FIRST,sun_sales#20 ASC NULLS FIRST,mon_sales#21 ASC NULLS FIRST,tue_sales#22 ASC NULLS FIRST,wed_sales#23 ASC NULLS FIRST,thu_sales#24 ASC NULLS FIRST,fri_sales#25 ASC NULLS FIRST,sat_sales#26 ASC NULLS FIRST], output=[s_store_name#10,s_store_id#12,sun_sales#20,mon_sales#21,tue_sales#22,wed_sales#23,thu_sales#24,fri_sales#25,sat_sales#26]), [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26], 100, 0, [s_store_name#10 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sun_sales#20 ASC NULLS FIRST, mon_sales#21 ASC NULLS FIRST, tue_sales#22 ASC NULLS FIRST, wed_sales#23 ASC NULLS FIRST, thu_sales#24 ASC NULLS FIRST, fri_sales#25 ASC NULLS FIRST, sat_sales#26 ASC NULLS FIRST], [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] + +(19) CometColumnarToRow [codegen id : 1] +Input [9]: [s_store_name#10, s_store_id#12, sun_sales#20, mon_sales#21, tue_sales#22, wed_sales#23, thu_sales#24, fri_sales#25, sat_sales#26] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt index 61dd9837e5..cb125e2456 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_name#1, s_store_id#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#4 END))#5,17,2) AS sun_sales#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#4 END))#7,17,2) AS mon_sales#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#4 END))#9,17,2) AS tue_sales#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#4 END))#11,17,2) AS wed_sales#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#4 END))#13,17,2) AS thu_sales#14, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#4 END))#15,17,2) AS fri_sales#16, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#4 END))#17,17,2) AS sat_sales#18)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt index ae850c9d3c..59ad7611a5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - WholeStageCodegen (1) - HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] CometExchange [s_store_name,s_store_id] #1 CometHashAggregate [d_day_name,ss_sales_price] [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] CometProject [d_day_name,ss_sales_price,s_store_id,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt index b92d33a9c5..08bba41c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/explain.txt @@ -1,47 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- 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 (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (30) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (37) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometHashAggregate (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- 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 (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) + +- ReusedExchange (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -173,110 +169,88 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 3] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] - -(29) HashAggregate [codegen id : 3] +(28) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#19 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) +(30) CometFilter +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) -(32) CometProject -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#32] +(31) CometProject +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#24, 20)) AS c_first_name#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#25, 30)) AS c_last_name#27] -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] +(32) CometBroadcastExchange +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -(34) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(33) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] +Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None +(34) CometProject +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -(36) Project [codegen id : 3] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] +(35) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#31, ca_city#32] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#33, ca_city#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#33, ca_city#34] -Condition : (isnotnull(ca_address_sk#33) AND isnotnull(ca_city#34)) - -(39) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#33, ca_city#34] +(36) CometBroadcastHashJoin +Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Right output [2]: [ca_address_sk#31, ca_city#32] +Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight -(40) BroadcastExchange -Input [2]: [ca_address_sk#33, ca_city#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(37) CometProject +Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] +Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -(41) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#24) +(38) CometTakeOrderedAndProject +Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -(42) Project [codegen id : 3] -Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] - -(43) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +(39) CometColumnarToRow [codegen id : 1] +Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(41) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(46) CometProject +(42) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#10] -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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt index a7ef24155f..c842000e11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/extended.txt @@ -1,51 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, ca_city#3 AS bought_city#4, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#5))#6,17,2) AS amt#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#9,17,2) AS profit#10)] - : : +- CometColumnarToRow - : : +- 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.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 35 out of 45 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt index 6754c4f519..e646cb620a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46.native_iceberg_compat/simplified.txt @@ -1,12 +1,12 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (3) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] @@ -42,18 +42,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt index b92d33a9c5..08bba41c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt @@ -1,47 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- 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 (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (30) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (37) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometHashAggregate (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- 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 (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) + +- ReusedExchange (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -173,110 +169,88 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 3] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] - -(29) HashAggregate [codegen id : 3] +(28) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19, sum#20, sum#21] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#19] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#19 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) +(30) CometFilter +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) -(32) CometProject -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] -Arguments: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32], [c_customer_sk#27, c_current_addr_sk#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#29, 20)) AS c_first_name#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#30, 30)) AS c_last_name#32] +(31) CometProject +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [c_customer_sk#22, c_current_addr_sk#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#24, 20)) AS c_first_name#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#25, 30)) AS c_last_name#27] -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] +(32) CometBroadcastExchange +Input [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -(34) BroadcastExchange -Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(33) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30] +Right output [4]: [c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [ss_customer_sk#1], [c_customer_sk#22], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#27] -Join type: Inner -Join condition: None +(34) CometProject +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, amt#29, profit#30, c_customer_sk#22, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Arguments: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27], [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] -(36) Project [codegen id : 3] -Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32] -Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#31, c_last_name#32] +(35) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#31, ca_city#32] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#33, ca_city#34] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#33, ca_city#34] -Condition : (isnotnull(ca_address_sk#33) AND isnotnull(ca_city#34)) - -(39) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#33, ca_city#34] +(36) CometBroadcastHashJoin +Left output [7]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27] +Right output [2]: [ca_address_sk#31, ca_city#32] +Arguments: [c_current_addr_sk#23], [ca_address_sk#31], Inner, NOT (ca_city#32 = bought_city#28), BuildRight -(40) BroadcastExchange -Input [2]: [ca_address_sk#33, ca_city#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(37) CometProject +Input [9]: [ss_ticket_number#5, bought_city#28, amt#29, profit#30, c_current_addr_sk#23, c_first_name#26, c_last_name#27, ca_address_sk#31, ca_city#32] +Arguments: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -(41) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_current_addr_sk#28] -Right keys [1]: [ca_address_sk#33] -Join type: Inner -Join condition: NOT (ca_city#34 = bought_city#24) +(38) CometTakeOrderedAndProject +Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#27 ASC NULLS FIRST,c_first_name#26 ASC NULLS FIRST,ca_city#32 ASC NULLS FIRST,bought_city#28 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#27,c_first_name#26,ca_city#32,bought_city#28,ss_ticket_number#5,amt#29,profit#30]), [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30], 100, 0, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#28 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] -(42) Project [codegen id : 3] -Output [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#31, c_last_name#32, ca_address_sk#33, ca_city#34] - -(43) TakeOrderedAndProject -Input [7]: [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] -Arguments: 100, [c_last_name#32 ASC NULLS FIRST, c_first_name#31 ASC NULLS FIRST, ca_city#34 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#32, c_first_name#31, ca_city#34, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +(39) CometColumnarToRow [codegen id : 1] +Input [7]: [c_last_name#27, c_first_name#26, ca_city#32, bought_city#28, ss_ticket_number#5, amt#29, profit#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(41) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Condition : ((d_dow#12 IN (6,0) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(46) CometProject +(42) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(47) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(48) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#10] -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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt index a7ef24155f..c842000e11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/extended.txt @@ -1,51 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, ca_city#3 AS bought_city#4, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#5))#6,17,2) AS amt#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#8))#9,17,2) AS profit#10)] - : : +- CometColumnarToRow - : : +- 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.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 35 out of 45 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt index 6754c4f519..e646cb620a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -1,12 +1,12 @@ -TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - WholeStageCodegen (3) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + CometBroadcastHashJoin [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,amt,profit,ca_city,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] @@ -42,18 +42,8 @@ TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_nu CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt index 52a3eee555..5b6a94ab57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] - -(22) HashAggregate [codegen id : 1] +(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))#18] -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))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#18,17,2) AS _w0#20] -(23) 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#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) 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#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(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) 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#19, _w0#20] -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#21], [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] -(27) Filter [codegen id : 3] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(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)) -(28) 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#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) 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#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(29) Filter [codegen id : 10] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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 [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] -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) HashAggregate [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] -Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] +(31) CometHashAggregate +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -(34) CometColumnarExchange -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 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] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -(37) Window -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, 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#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +(36) Project [codegen id : 4] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] -(39) BroadcastExchange -Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(37) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 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#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, 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#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(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#30] +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#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 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] -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, 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#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(44) Project [codegen id : 6] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(45) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 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#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(47) 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, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 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#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -(50) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(48) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt index 7c578b0474..b50b570b4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt index 0c8d77b89c..19124a9e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) + WholeStageCodegen (7) Project [i_category,i_brand,s_store_name,s_company_name,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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - 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 - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (3) 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] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index 52a3eee555..5b6a94ab57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] - -(22) HashAggregate [codegen id : 1] +(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))#18] -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))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#18,17,2) AS _w0#20] -(23) 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#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) 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#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(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) 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#19, _w0#20] -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#21], [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] -(27) Filter [codegen id : 3] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(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)) -(28) 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#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) 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#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(29) Filter [codegen id : 10] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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 [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] -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) HashAggregate [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] -Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] +(31) CometHashAggregate +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -(34) CometColumnarExchange -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 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] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -(37) Window -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, 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#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +(36) Project [codegen id : 4] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] -(39) BroadcastExchange -Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(37) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 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#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, 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#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(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#30] +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#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 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] -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, 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#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(44) Project [codegen id : 6] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(45) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 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#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(47) 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, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 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#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -(50) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(48) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt index 7c578b0474..b50b570b4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 0c8d77b89c..19124a9e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) + WholeStageCodegen (7) Project [i_category,i_brand,s_store_name,s_company_name,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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_bra Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - 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 - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (3) 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] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt index 03c4461bcd..6f23a02597 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (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) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(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))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(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) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(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) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(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] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(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] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(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) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(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#62, cp_catalog_page_id#63] +(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#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#63, 16)) AS cp_catalog_page_id#64] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(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] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(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 -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(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] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(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))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(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] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(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))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(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) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(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] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(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 -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(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)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(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] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(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 -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#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] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#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] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(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 -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(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 -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) 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) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#106, 16)) AS web_site_id#107] +(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) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(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) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(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) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(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]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +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]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 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-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt index f3e7cf1b12..d128acaf06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/extended.txt @@ -1,93 +1,90 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#1))#2,17,2) AS sales#3, MakeDecimal(sum(UnscaledValue(return_amt#4))#5,17,2) AS returns#6, (MakeDecimal(sum(UnscaledValue(profit#7))#8,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#9))#10,17,2)) AS profit#11, store channel AS channel#12, concat(store, s_store_id#13) AS id#14)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#15))#16,17,2) AS sales#17, MakeDecimal(sum(UnscaledValue(return_amt#18))#19,17,2) AS returns#20, (MakeDecimal(sum(UnscaledValue(profit#21))#22,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#23))#24,17,2)) AS profit#25, catalog channel AS channel#26, concat(catalog_page, cp_catalog_page_id#27) AS id#28)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(return_amt#32))#33,17,2) AS returns#34, (MakeDecimal(sum(UnscaledValue(profit#35))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#37))#38,17,2)) AS profit#39, web channel AS channel#40, concat(web_site, web_site_id#41) AS id#42)] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 71 out of 86 eligible operators (82%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5.native_iceberg_compat/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - 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 #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] - 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] #4 - 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] #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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - 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] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - 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 (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - 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] #9 - 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] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - 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 (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + 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 #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] + 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] #4 + 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] #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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + 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] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + 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] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + 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] #9 + 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] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt index 03c4461bcd..6f23a02597 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (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) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(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))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(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) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(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) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(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] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(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] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(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) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(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#62, cp_catalog_page_id#63] +(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#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#63, 16)) AS cp_catalog_page_id#64] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(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] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(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 -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(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] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(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))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(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] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(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))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(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) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(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] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(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 -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(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)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(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] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(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 -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#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] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#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] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(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 -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(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 -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) 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) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#106, 16)) AS web_site_id#107] +(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) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(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) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(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) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(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]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +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]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 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-v1_4-spark4_0/q5/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt index f3e7cf1b12..d128acaf06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/extended.txt @@ -1,93 +1,90 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#1))#2,17,2) AS sales#3, MakeDecimal(sum(UnscaledValue(return_amt#4))#5,17,2) AS returns#6, (MakeDecimal(sum(UnscaledValue(profit#7))#8,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#9))#10,17,2)) AS profit#11, store channel AS channel#12, concat(store, s_store_id#13) AS id#14)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#15))#16,17,2) AS sales#17, MakeDecimal(sum(UnscaledValue(return_amt#18))#19,17,2) AS returns#20, (MakeDecimal(sum(UnscaledValue(profit#21))#22,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#23))#24,17,2)) AS profit#25, catalog channel AS channel#26, concat(catalog_page, cp_catalog_page_id#27) AS id#28)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(return_amt#32))#33,17,2) AS returns#34, (MakeDecimal(sum(UnscaledValue(profit#35))#36,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#37))#38,17,2)) AS profit#39, web channel AS channel#40, concat(web_site, web_site_id#41) AS id#42)] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 71 out of 86 eligible operators (82%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 79 out of 86 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - 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 #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] - 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] #4 - 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] #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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - 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] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - 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 (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - 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] #9 - 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] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - 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 (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + 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 #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] + 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] #4 + 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] #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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + 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] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + 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] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + 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] #9 + 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] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt index bb1b8f3167..683cc25d24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Filter (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometProject (37) - +- CometSortMergeJoin (36) - :- CometSort (19) - : +- CometColumnarExchange (18) - : +- * Project (17) - : +- Window (16) - : +- * CometColumnarToRow (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (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) - +- CometSort (35) - +- CometColumnarExchange (34) - +- * Project (33) - +- Window (32) - +- * CometColumnarToRow (31) - +- CometSort (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) - +- ReusedExchange (22) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * 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) + +- CometSort (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -93,177 +91,167 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(12) HashAggregate [codegen id : 1] +(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 _w0#11, ws_item_sk#1] -(13) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, _w0#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, _w0#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(14) CometSort -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#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, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +(14) CometColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -(16) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(17) Project [codegen id : 3] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] +Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -(18) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(17) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(18) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#17, d_date#18] +(20) CometFilter +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_item_sk#12) -(23) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Right output [2]: [d_date_sk#17, d_date#18] -Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight +(21) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#16, d_date#17] -(24) CometProject -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] -Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] +(22) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(25) CometHashAggregate -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +(23) CometProject +Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] +Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(26) CometExchange -Input [3]: [ss_item_sk#13, d_date#18, sum#19] -Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(24) CometHashAggregate +Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] +(25) CometExchange +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] +(26) CometHashAggregate +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(29) CometColumnarExchange -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(27) CometExchange +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(30) CometSort -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] +(28) CometSort +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(31) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] +(29) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -(32) Window -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +(30) Window +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(33) Project [codegen id : 6] -Output [3]: [item_sk#21, d_date#18, cume_sales#23] -Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] +(31) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] +Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -(34) CometColumnarExchange -Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(32) CometColumnarExchange +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) CometSort -Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] +(33) CometSort +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(36) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter +(34) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#11] +Right output [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(37) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] +(35) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(38) CometExchange -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(36) CometExchange +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] +(37) CometSort +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +(38) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -(41) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] +(39) Window +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(42) Filter [codegen id : 8] -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) +(40) Filter [codegen id : 6] +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(43) TakeOrderedAndProject -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +(41) TakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== 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 (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) 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,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(43) 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 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject +(44) 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] -(47) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(48) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt index ccdffa0dc0..66c5717cc7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt @@ -12,44 +12,42 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- 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 + : +- 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 +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- 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 + +- 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 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt index dc6a7ea61f..a5af2e114b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (7) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] @@ -12,57 +12,51 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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 (3) + WholeStageCodegen (2) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - 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 #5 - 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] #6 - 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] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + 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 #5 + 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] #6 + 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] CometSort [item_sk,d_date,cume_sales] CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (4) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - 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] #6 + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + 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] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index bb1b8f3167..683cc25d24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Filter (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometProject (37) - +- CometSortMergeJoin (36) - :- CometSort (19) - : +- CometColumnarExchange (18) - : +- * Project (17) - : +- Window (16) - : +- * CometColumnarToRow (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (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) - +- CometSort (35) - +- CometColumnarExchange (34) - +- * Project (33) - +- Window (32) - +- * CometColumnarToRow (31) - +- CometSort (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) - +- ReusedExchange (22) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * 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) + +- CometSort (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -93,177 +91,167 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(12) HashAggregate [codegen id : 1] +(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 _w0#11, ws_item_sk#1] -(13) CometColumnarExchange -Input [4]: [item_sk#10, d_date#6, _w0#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, _w0#10, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(14) CometSort -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#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, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +(14) CometColumnarToRow [codegen id : 1] +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -(16) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] +(15) Window +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(17) Project [codegen id : 3] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] -Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] +Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -(18) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(17) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(19) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(18) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(21) CometFilter -Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Condition : isnotnull(ss_item_sk#13) - -(22) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#17, d_date#18] +(20) CometFilter +Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_item_sk#12) -(23) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] -Right output [2]: [d_date_sk#17, d_date#18] -Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight +(21) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#16, d_date#17] -(24) CometProject -Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] -Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] +(22) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] +Right output [2]: [d_date_sk#16, d_date#17] +Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(25) CometHashAggregate -Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +(23) CometProject +Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] +Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(26) CometExchange -Input [3]: [ss_item_sk#13, d_date#18, sum#19] -Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(24) CometHashAggregate +Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] +(25) CometExchange +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(28) HashAggregate [codegen id : 4] -Input [3]: [ss_item_sk#13, d_date#18, sum#19] -Keys [2]: [ss_item_sk#13, d_date#18] -Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] -Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] +(26) CometHashAggregate +Input [3]: [ss_item_sk#12, d_date#17, sum#18] +Keys [2]: [ss_item_sk#12, d_date#17] +Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(29) CometColumnarExchange -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(27) CometExchange +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(30) CometSort -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] +(28) CometSort +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(31) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] +(29) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -(32) Window -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +(30) Window +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(33) Project [codegen id : 6] -Output [3]: [item_sk#21, d_date#18, cume_sales#23] -Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] +(31) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] +Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -(34) CometColumnarExchange -Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +(32) CometColumnarExchange +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(35) CometSort -Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] +(33) CometSort +Input [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(36) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#12] -Right output [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter +(34) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#11] +Right output [3]: [item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(37) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] +(35) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(38) CometExchange -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(36) CometExchange +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] +(37) CometSort +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +(38) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -(41) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] +(39) Window +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(42) Filter [codegen id : 8] -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) +(40) Filter [codegen id : 6] +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(43) TakeOrderedAndProject -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +(41) TakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== 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 (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) 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,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(43) 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 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject +(44) 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] -(47) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(48) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt index ccdffa0dc0..66c5717cc7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt @@ -12,44 +12,42 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- 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 + : +- 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 +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- 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 + +- 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 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index dc6a7ea61f..a5af2e114b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -1,9 +1,9 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (8) + WholeStageCodegen (6) Filter [web_cumulative,store_cumulative] InputAdapter Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (7) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] @@ -12,57 +12,51 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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 (3) + WholeStageCodegen (2) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - 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 #5 - 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] #6 - 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] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + 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 #5 + 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] #6 + 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] CometSort [item_sk,d_date,cume_sales] CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (6) + WholeStageCodegen (4) Project [item_sk,d_date,cume_sales] InputAdapter Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (4) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - 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] #6 + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + 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] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt index f95e491037..68a8ab88fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt index aa974dceb8..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2 AS brand_id#3, i_brand#4 AS brand#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS ext_price#8)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt index 05bbc57061..218062c3ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] + CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt index f95e491037..68a8ab88fa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(d_year#2, i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [4]: [d_year#2, i_brand#11, i_brand_id#8, sum#12] Keys [3]: [d_year#2, i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(19) TakeOrderedAndProject -Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] -Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] +(18) CometTakeOrderedAndProject +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[d_year#2 ASC NULLS FIRST,ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[d_year#2,brand_id#13,brand#14,ext_price#15]), [d_year#2, brand_id#13, brand#14, ext_price#15], 100, 0, [d_year#2 ASC NULLS FIRST, ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [d_year#2, brand_id#13, brand#14, ext_price#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [4]: [d_year#2, brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt index aa974dceb8..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2 AS brand_id#3, i_brand#4 AS brand#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS ext_price#8)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt index 05bbc57061..218062c3ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [d_year,ext_price,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [d_year,brand_id,brand,ext_price] + CometHashAggregate [sum] [d_year,brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [d_year,i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [d_year,i_brand,i_brand_id,sum] CometProject [d_year,ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt index 474ac8078d..5b68f4c00c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] +(24) CometSort +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] +(26) Window +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(28) Filter [codegen id : 3] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 2] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +(28) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#15, d_qoy#17] 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-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt index 8351f359bc..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt index eab0efef6c..0c123eb728 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + 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 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index 474ac8078d..5b68f4c00c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] Arguments: hashpartitioning(i_manufact_id#5, d_qoy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [3]: [i_manufact_id#5, d_qoy#17, sum#19] Keys [2]: [i_manufact_id#5, d_qoy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] +(24) CometSort +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] +(26) Window +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(28) Filter [codegen id : 3] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 2] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +(28) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -(30) TakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#15, d_month_seq#16, d_qoy#17] Arguments: [d_date_sk#15, d_qoy#17], [d_date_sk#15, d_qoy#17] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_qoy#17] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#15, d_qoy#17] 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-v1_4-spark4_0/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt index 8351f359bc..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index eab0efef6c..0c123eb728 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_manufact_id,sum_sales,avg_quarterly_sales] Filter [avg_quarterly_sales,sum_sales] InputAdapter Window [_w0,i_manufact_id] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - 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 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + 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 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt index c03c077632..9eb7865391 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/explain.txt @@ -1,63 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * HashAggregate (58) - +- * CometColumnarToRow (57) - +- CometColumnarExchange (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * CometColumnarToRow (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +* CometColumnarToRow (58) ++- CometTakeOrderedAndProject (57) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometHashAggregate (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometProject (50) + +- CometBroadcastHashJoin (49) + :- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometExchange (26) + : : : : +- CometHashAggregate (25) + : : : : +- CometProject (24) + : : : : +- CometBroadcastHashJoin (23) + : : : : :- CometProject (19) + : : : : : +- CometBroadcastHashJoin (18) + : : : : : :- CometProject (13) + : : : : : : +- CometBroadcastHashJoin (12) + : : : : : : :- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) + : : : : : : +- CometBroadcastExchange (11) + : : : : : : +- CometProject (10) + : : : : : : +- CometFilter (9) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) + : : : : : +- CometBroadcastExchange (17) + : : : : : +- CometProject (16) + : : : : : +- CometFilter (15) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) + : : : : +- CometBroadcastExchange (22) + : : : : +- CometFilter (21) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (30) + : : : +- CometFilter (29) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) + : +- CometBroadcastExchange (42) + : +- CometProject (41) + : +- CometFilter (40) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) + +- CometBroadcastExchange (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -308,192 +307,183 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] Input [2]: [c_customer_sk#20, sum#39] Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(53) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#20, sum#39] - -(54) HashAggregate [codegen id : 1] +(53) CometHashAggregate Input [2]: [c_customer_sk#20, sum#39] Keys [1]: [c_customer_sk#20] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#23))#40] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#23))#40,17,2) / 50) as int) AS segment#41] -(55) HashAggregate [codegen id : 1] -Input [1]: [segment#41] -Keys [1]: [segment#41] +(54) CometHashAggregate +Input [1]: [segment#40] +Keys [1]: [segment#40] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#42] -Results [2]: [segment#41, count#43] -(56) CometColumnarExchange -Input [2]: [segment#41, count#43] -Arguments: hashpartitioning(segment#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(55) CometExchange +Input [2]: [segment#40, count#41] +Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(57) CometColumnarToRow [codegen id : 2] -Input [2]: [segment#41, count#43] - -(58) HashAggregate [codegen id : 2] -Input [2]: [segment#41, count#43] -Keys [1]: [segment#41] +(56) CometHashAggregate +Input [2]: [segment#40, count#41] +Keys [1]: [segment#40] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#44] -Results [3]: [segment#41, count(1)#44 AS num_customers#45, (segment#41 * 50) AS segment_base#46] -(59) TakeOrderedAndProject -Input [3]: [segment#41, num_customers#45, segment_base#46] -Arguments: 100, [segment#41 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#41, num_customers#45, segment_base#46] +(57) CometTakeOrderedAndProject +Input [3]: [segment#40, num_customers#42, segment_base#43] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] + +(58) CometColumnarToRow [codegen id : 1] +Input [3]: [segment#40, num_customers#42, segment_base#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (60) +BroadcastExchange (63) ++- * CometColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#17, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter +(60) CometFilter Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(62) CometProject +(61) CometProject Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(63) CometColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(64) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (68) ++- * CometColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#33, d_month_seq#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter +(65) CometFilter Input [2]: [d_date_sk#33, d_month_seq#34] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) -(67) CometProject +(66) CometProject Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(68) CometColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#33] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:4 Hosting operator id = 66 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] +Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] -Subquery:5 Hosting operator id = 66 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] +Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] -Subquery:6 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (76) -+- CometHashAggregate (75) - +- CometExchange (74) - +- CometHashAggregate (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] +* CometColumnarToRow (75) ++- CometHashAggregate (74) + +- CometExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#47, d_year#48, d_moy#49] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#44, d_year#45, d_moy#46] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(71) CometFilter -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Condition : (((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 1998)) AND (d_moy#49 = 12)) +(70) CometFilter +Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) -(72) CometProject -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Arguments: [(d_month_seq + 1)#50], [(d_month_seq#47 + 1) AS (d_month_seq + 1)#50] +(71) CometProject +Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] -(73) CometHashAggregate -Input [1]: [(d_month_seq + 1)#50] -Keys [1]: [(d_month_seq + 1)#50] +(72) CometHashAggregate +Input [1]: [(d_month_seq + 1)#47] +Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(74) CometExchange -Input [1]: [(d_month_seq + 1)#50] -Arguments: hashpartitioning((d_month_seq + 1)#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(73) CometExchange +Input [1]: [(d_month_seq + 1)#47] +Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#50] -Keys [1]: [(d_month_seq + 1)#50] +(74) CometHashAggregate +Input [1]: [(d_month_seq + 1)#47] +Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(76) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#50] +(75) CometColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#47] -Subquery:7 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (83) -+- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] +* CometColumnarToRow (82) ++- CometHashAggregate (81) + +- CometExchange (80) + +- CometHashAggregate (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#51, d_year#52, d_moy#53] +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#48, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(78) CometFilter -Input [3]: [d_month_seq#51, d_year#52, d_moy#53] -Condition : (((isnotnull(d_year#52) AND isnotnull(d_moy#53)) AND (d_year#52 = 1998)) AND (d_moy#53 = 12)) +(77) CometFilter +Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) -(79) CometProject -Input [3]: [d_month_seq#51, d_year#52, d_moy#53] -Arguments: [(d_month_seq + 3)#54], [(d_month_seq#51 + 3) AS (d_month_seq + 3)#54] +(78) CometProject +Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] -(80) CometHashAggregate -Input [1]: [(d_month_seq + 3)#54] -Keys [1]: [(d_month_seq + 3)#54] +(79) CometHashAggregate +Input [1]: [(d_month_seq + 3)#51] +Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(81) CometExchange -Input [1]: [(d_month_seq + 3)#54] -Arguments: hashpartitioning((d_month_seq + 3)#54, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(80) CometExchange +Input [1]: [(d_month_seq + 3)#51] +Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#54] -Keys [1]: [(d_month_seq + 3)#54] +(81) CometHashAggregate +Input [1]: [(d_month_seq + 3)#51] +Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(83) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#54] +(82) CometColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#51] Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt index e5227bd33f..a49bbabd10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/extended.txt @@ -1,96 +1,95 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported result expressions found in: List(cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) / 50) as int) AS segment#3)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- 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 - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- 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 - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- 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 + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- 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 + : : : : :- ReusedSubquery + : : : : +- ReusedSubquery + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- 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.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- ReusedSubquery + +- ReusedSubquery -Comet accelerated 71 out of 88 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt index 027968b881..ce1acded44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54.native_iceberg_compat/simplified.txt @@ -1,106 +1,103 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (2) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - WholeStageCodegen (1) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_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] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - 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 [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] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - 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_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [segment,num_customers,segment_base] + CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] + CometExchange [segment] #1 + CometHashAggregate [segment,count] + CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [c_customer_sk] #2 + CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] + CometProject [c_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] + CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometExchange [c_customer_sk,c_current_addr_sk] #3 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - 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] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - 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] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #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] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [i_item_sk] #5 + CometProject [i_item_sk] + 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 [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] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + 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_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + 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] + Subquery #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + 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] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 + CometProject [ca_state] [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometBroadcastExchange [s_county,s_state] #13 + CometProject [s_state] [s_county,s_state] + CometFilter [s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt index c03c077632..9eb7865391 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt @@ -1,63 +1,62 @@ == Physical Plan == -TakeOrderedAndProject (59) -+- * HashAggregate (58) - +- * CometColumnarToRow (57) - +- CometColumnarExchange (56) - +- * HashAggregate (55) - +- * HashAggregate (54) - +- * CometColumnarToRow (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometProject (50) - +- CometBroadcastHashJoin (49) - :- CometProject (44) - : +- CometBroadcastHashJoin (43) - : :- CometProject (38) - : : +- CometBroadcastHashJoin (37) - : : :- CometProject (32) - : : : +- CometBroadcastHashJoin (31) - : : : :- CometHashAggregate (27) - : : : : +- CometExchange (26) - : : : : +- CometHashAggregate (25) - : : : : +- CometProject (24) - : : : : +- CometBroadcastHashJoin (23) - : : : : :- CometProject (19) - : : : : : +- CometBroadcastHashJoin (18) - : : : : : :- CometProject (13) - : : : : : : +- CometBroadcastHashJoin (12) - : : : : : : :- CometUnion (7) - : : : : : : : :- CometProject (3) - : : : : : : : : +- CometFilter (2) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) - : : : : : : : +- CometProject (6) - : : : : : : : +- CometFilter (5) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) - : : : : : : +- CometBroadcastExchange (11) - : : : : : : +- CometProject (10) - : : : : : : +- CometFilter (9) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) - : : : : : +- CometBroadcastExchange (17) - : : : : : +- CometProject (16) - : : : : : +- CometFilter (15) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) - : : : : +- CometBroadcastExchange (22) - : : : : +- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) - : : : +- CometBroadcastExchange (30) - : : : +- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- CometBroadcastExchange (36) - : : +- CometProject (35) - : : +- CometFilter (34) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) - : +- CometBroadcastExchange (42) - : +- CometProject (41) - : +- CometFilter (40) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) - +- CometBroadcastExchange (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +* CometColumnarToRow (58) ++- CometTakeOrderedAndProject (57) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometHashAggregate (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometProject (50) + +- CometBroadcastHashJoin (49) + :- CometProject (44) + : +- CometBroadcastHashJoin (43) + : :- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometHashAggregate (27) + : : : : +- CometExchange (26) + : : : : +- CometHashAggregate (25) + : : : : +- CometProject (24) + : : : : +- CometBroadcastHashJoin (23) + : : : : :- CometProject (19) + : : : : : +- CometBroadcastHashJoin (18) + : : : : : :- CometProject (13) + : : : : : : +- CometBroadcastHashJoin (12) + : : : : : : :- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (4) + : : : : : : +- CometBroadcastExchange (11) + : : : : : : +- CometProject (10) + : : : : : : +- CometFilter (9) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (8) + : : : : : +- CometBroadcastExchange (17) + : : : : : +- CometProject (16) + : : : : : +- CometFilter (15) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (14) + : : : : +- CometBroadcastExchange (22) + : : : : +- CometFilter (21) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (20) + : : : +- CometBroadcastExchange (30) + : : : +- CometFilter (29) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (33) + : +- CometBroadcastExchange (42) + : +- CometProject (41) + : +- CometFilter (40) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (39) + +- CometBroadcastExchange (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales @@ -308,192 +307,183 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#23))] Input [2]: [c_customer_sk#20, sum#39] Arguments: hashpartitioning(c_customer_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(53) CometColumnarToRow [codegen id : 1] -Input [2]: [c_customer_sk#20, sum#39] - -(54) HashAggregate [codegen id : 1] +(53) CometHashAggregate Input [2]: [c_customer_sk#20, sum#39] Keys [1]: [c_customer_sk#20] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#23))#40] -Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#23))#40,17,2) / 50) as int) AS segment#41] -(55) HashAggregate [codegen id : 1] -Input [1]: [segment#41] -Keys [1]: [segment#41] +(54) CometHashAggregate +Input [1]: [segment#40] +Keys [1]: [segment#40] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#42] -Results [2]: [segment#41, count#43] -(56) CometColumnarExchange -Input [2]: [segment#41, count#43] -Arguments: hashpartitioning(segment#41, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(55) CometExchange +Input [2]: [segment#40, count#41] +Arguments: hashpartitioning(segment#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(57) CometColumnarToRow [codegen id : 2] -Input [2]: [segment#41, count#43] - -(58) HashAggregate [codegen id : 2] -Input [2]: [segment#41, count#43] -Keys [1]: [segment#41] +(56) CometHashAggregate +Input [2]: [segment#40, count#41] +Keys [1]: [segment#40] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#44] -Results [3]: [segment#41, count(1)#44 AS num_customers#45, (segment#41 * 50) AS segment_base#46] -(59) TakeOrderedAndProject -Input [3]: [segment#41, num_customers#45, segment_base#46] -Arguments: 100, [segment#41 ASC NULLS FIRST, num_customers#45 ASC NULLS FIRST], [segment#41, num_customers#45, segment_base#46] +(57) CometTakeOrderedAndProject +Input [3]: [segment#40, num_customers#42, segment_base#43] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[segment#40 ASC NULLS FIRST,num_customers#42 ASC NULLS FIRST], output=[segment#40,num_customers#42,segment_base#43]), [segment#40, num_customers#42, segment_base#43], 100, 0, [segment#40 ASC NULLS FIRST, num_customers#42 ASC NULLS FIRST], [segment#40, num_customers#42, segment_base#43] + +(58) CometColumnarToRow [codegen id : 1] +Input [3]: [segment#40, num_customers#42, segment_base#43] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometFilter (61) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (60) +BroadcastExchange (63) ++- * CometColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (59) -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(59) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#17, d_year#18, d_moy#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct -(61) CometFilter +(60) CometFilter Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) -(62) CometProject +(61) CometProject Input [3]: [d_date_sk#17, d_year#18, d_moy#19] Arguments: [d_date_sk#17], [d_date_sk#17] -(63) CometColumnarToRow [codegen id : 1] +(62) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#17] -(64) BroadcastExchange +(63) BroadcastExchange Input [1]: [d_date_sk#17] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#24 IN dynamicpruning#25 -BroadcastExchange (69) -+- * CometColumnarToRow (68) - +- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +BroadcastExchange (68) ++- * CometColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (64) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(64) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#33, d_month_seq#34] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#35), LessThanOrEqual(d_month_seq,ScalarSubquery#36), IsNotNull(d_date_sk)] ReadSchema: struct -(66) CometFilter +(65) CometFilter Input [2]: [d_date_sk#33, d_month_seq#34] Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= ReusedSubquery Subquery scalar-subquery#35, [id=#37])) AND (d_month_seq#34 <= ReusedSubquery Subquery scalar-subquery#36, [id=#38])) AND isnotnull(d_date_sk#33)) -(67) CometProject +(66) CometProject Input [2]: [d_date_sk#33, d_month_seq#34] Arguments: [d_date_sk#33], [d_date_sk#33] -(68) CometColumnarToRow [codegen id : 1] +(67) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#33] -(69) BroadcastExchange +(68) BroadcastExchange Input [1]: [d_date_sk#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:4 Hosting operator id = 66 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] +Subquery:4 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] -Subquery:5 Hosting operator id = 66 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] +Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#36, [id=#38] -Subquery:6 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#35, [id=#37] -* CometColumnarToRow (76) -+- CometHashAggregate (75) - +- CometExchange (74) - +- CometHashAggregate (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +Subquery:6 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#35, [id=#37] +* CometColumnarToRow (75) ++- CometHashAggregate (74) + +- CometExchange (73) + +- CometHashAggregate (72) + +- CometProject (71) + +- CometFilter (70) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (69) -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#47, d_year#48, d_moy#49] +(69) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#44, d_year#45, d_moy#46] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(71) CometFilter -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Condition : (((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 1998)) AND (d_moy#49 = 12)) +(70) CometFilter +Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +Condition : (((isnotnull(d_year#45) AND isnotnull(d_moy#46)) AND (d_year#45 = 1998)) AND (d_moy#46 = 12)) -(72) CometProject -Input [3]: [d_month_seq#47, d_year#48, d_moy#49] -Arguments: [(d_month_seq + 1)#50], [(d_month_seq#47 + 1) AS (d_month_seq + 1)#50] +(71) CometProject +Input [3]: [d_month_seq#44, d_year#45, d_moy#46] +Arguments: [(d_month_seq + 1)#47], [(d_month_seq#44 + 1) AS (d_month_seq + 1)#47] -(73) CometHashAggregate -Input [1]: [(d_month_seq + 1)#50] -Keys [1]: [(d_month_seq + 1)#50] +(72) CometHashAggregate +Input [1]: [(d_month_seq + 1)#47] +Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(74) CometExchange -Input [1]: [(d_month_seq + 1)#50] -Arguments: hashpartitioning((d_month_seq + 1)#50, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(73) CometExchange +Input [1]: [(d_month_seq + 1)#47] +Arguments: hashpartitioning((d_month_seq + 1)#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(75) CometHashAggregate -Input [1]: [(d_month_seq + 1)#50] -Keys [1]: [(d_month_seq + 1)#50] +(74) CometHashAggregate +Input [1]: [(d_month_seq + 1)#47] +Keys [1]: [(d_month_seq + 1)#47] Functions: [] -(76) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 1)#50] +(75) CometColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#47] -Subquery:7 Hosting operator id = 65 Hosting Expression = Subquery scalar-subquery#36, [id=#38] -* CometColumnarToRow (83) -+- CometHashAggregate (82) - +- CometExchange (81) - +- CometHashAggregate (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +Subquery:7 Hosting operator id = 64 Hosting Expression = Subquery scalar-subquery#36, [id=#38] +* CometColumnarToRow (82) ++- CometHashAggregate (81) + +- CometExchange (80) + +- CometHashAggregate (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [3]: [d_month_seq#51, d_year#52, d_moy#53] +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#48, d_year#49, d_moy#50] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(78) CometFilter -Input [3]: [d_month_seq#51, d_year#52, d_moy#53] -Condition : (((isnotnull(d_year#52) AND isnotnull(d_moy#53)) AND (d_year#52 = 1998)) AND (d_moy#53 = 12)) +(77) CometFilter +Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +Condition : (((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 12)) -(79) CometProject -Input [3]: [d_month_seq#51, d_year#52, d_moy#53] -Arguments: [(d_month_seq + 3)#54], [(d_month_seq#51 + 3) AS (d_month_seq + 3)#54] +(78) CometProject +Input [3]: [d_month_seq#48, d_year#49, d_moy#50] +Arguments: [(d_month_seq + 3)#51], [(d_month_seq#48 + 3) AS (d_month_seq + 3)#51] -(80) CometHashAggregate -Input [1]: [(d_month_seq + 3)#54] -Keys [1]: [(d_month_seq + 3)#54] +(79) CometHashAggregate +Input [1]: [(d_month_seq + 3)#51] +Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(81) CometExchange -Input [1]: [(d_month_seq + 3)#54] -Arguments: hashpartitioning((d_month_seq + 3)#54, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(80) CometExchange +Input [1]: [(d_month_seq + 3)#51] +Arguments: hashpartitioning((d_month_seq + 3)#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(82) CometHashAggregate -Input [1]: [(d_month_seq + 3)#54] -Keys [1]: [(d_month_seq + 3)#54] +(81) CometHashAggregate +Input [1]: [(d_month_seq + 3)#51] +Keys [1]: [(d_month_seq + 3)#51] Functions: [] -(83) CometColumnarToRow [codegen id : 1] -Input [1]: [(d_month_seq + 3)#54] +(82) CometColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#51] Subquery:8 Hosting operator id = 46 Hosting Expression = ReusedSubquery Subquery scalar-subquery#35, [id=#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt index e5227bd33f..a49bbabd10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/extended.txt @@ -1,96 +1,95 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: Unsupported result expressions found in: List(cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) / 50) as int) AS segment#3)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometBroadcastHashJoin - : : : : : : :- CometUnion - : : : : : : : :- CometProject - : : : : : : : : +- 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 - : : : : : : : +- CometProject - : : : : : : : +- CometFilter - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : : : : +- ReusedSubquery - : : : : : : +- CometBroadcastExchange - : : : : : : +- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : : +- CometBroadcastExchange - : : : : : +- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- 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 - : : : : :- ReusedSubquery - : : : : +- ReusedSubquery - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : :- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- 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.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - : :- ReusedSubquery - : +- ReusedSubquery - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- ReusedSubquery - +- ReusedSubquery +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometBroadcastHashJoin + : : : : : : :- CometUnion + : : : : : : : :- CometProject + : : : : : : : : +- CometFilter + : : : : : : : : +- 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 + : : : : : : : +- CometProject + : : : : : : : +- CometFilter + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : : : : +- ReusedSubquery + : : : : : : +- CometBroadcastExchange + : : : : : : +- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : : +- CometBroadcastExchange + : : : : : +- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- 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 + : : : : :- ReusedSubquery + : : : : +- ReusedSubquery + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : :- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- 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.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + : :- ReusedSubquery + : +- ReusedSubquery + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- ReusedSubquery + +- ReusedSubquery -Comet accelerated 71 out of 88 eligible operators (80%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 88 eligible operators (85%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt index 027968b881..ce1acded44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -1,106 +1,103 @@ -TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (2) - HashAggregate [segment,count] [count(1),num_customers,segment_base,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [segment] #1 - WholeStageCodegen (1) - HashAggregate [segment] [count,count] - HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - CometColumnarToRow - InputAdapter - CometExchange [c_customer_sk] #2 - CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] - CometProject [c_customer_sk,ss_ext_sales_price] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] - CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] - CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometExchange [c_customer_sk,c_current_addr_sk] #3 - CometHashAggregate [c_customer_sk,c_current_addr_sk] - CometProject [c_customer_sk,c_current_addr_sk] - CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] - CometProject [customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] - CometProject [sold_date_sk,customer_sk] - CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] - CometUnion [sold_date_sk,customer_sk,item_sk] - CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_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] - CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] - CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [i_item_sk] #5 - CometProject [i_item_sk] - 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 [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] - CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 - 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_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 - CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #2 - BroadcastExchange #9 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - Subquery #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [segment,num_customers,segment_base] + CometHashAggregate [count] [segment,num_customers,segment_base,count(1)] + CometExchange [segment] #1 + CometHashAggregate [segment,count] + CometHashAggregate [c_customer_sk,sum] [segment,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [c_customer_sk] #2 + CometHashAggregate [ss_ext_sales_price] [c_customer_sk,sum] + CometProject [c_customer_sk,ss_ext_sales_price] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state,s_county,s_state] + CometProject [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk,ca_address_sk,ca_county,ca_state] + CometProject [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [c_customer_sk,c_current_addr_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometExchange [c_customer_sk,c_current_addr_sk] #3 + CometHashAggregate [c_customer_sk,c_current_addr_sk] + CometProject [c_customer_sk,c_current_addr_sk] + CometBroadcastHashJoin [customer_sk,c_customer_sk,c_current_addr_sk] + CometProject [customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,d_date_sk] + CometProject [sold_date_sk,customer_sk] + CometBroadcastHashJoin [sold_date_sk,customer_sk,item_sk,i_item_sk] + CometUnion [sold_date_sk,customer_sk,item_sk] + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 WholeStageCodegen (1) CometColumnarToRow InputAdapter - CometHashAggregate [(d_month_seq + 1)] - CometExchange [(d_month_seq + 1)] #10 - CometHashAggregate [(d_month_seq + 1)] - CometProject [d_month_seq] [(d_month_seq + 1)] - 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] - Subquery #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [(d_month_seq + 3)] - CometExchange [(d_month_seq + 3)] #11 - CometHashAggregate [(d_month_seq + 3)] - CometProject [d_month_seq] [(d_month_seq + 3)] - 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] - CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 - CometProject [ca_state] [ca_address_sk,ca_county,ca_state] - CometFilter [ca_address_sk,ca_county,ca_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] - CometBroadcastExchange [s_county,s_state] #13 - CometProject [s_state] [s_county,s_state] - CometFilter [s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] - CometBroadcastExchange [d_date_sk] #14 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #4 - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - ReusedSubquery [(d_month_seq + 1)] #3 - ReusedSubquery [(d_month_seq + 3)] #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] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [i_item_sk] #5 + CometProject [i_item_sk] + 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 [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] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk] #7 + 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_customer_sk,ss_ext_sales_price,ss_sold_date_sk] #8 + CometFilter [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #9 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometExchange [(d_month_seq + 1)] #10 + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + 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] + Subquery #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometExchange [(d_month_seq + 3)] #11 + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + 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] + CometBroadcastExchange [ca_address_sk,ca_county,ca_state] #12 + CometProject [ca_state] [ca_address_sk,ca_county,ca_state] + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + CometBroadcastExchange [s_county,s_state] #13 + CometProject [s_state] [s_county,s_state] + CometFilter [s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_county,s_state] + CometBroadcastExchange [d_date_sk] #14 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt index 801934db83..08e1bae2ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [3]: [i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [3]: [i_brand#11, i_brand_id#8, sum#12] Keys [2]: [i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(19) TakeOrderedAndProject -Input [3]: [brand_id#14, brand#15, ext_price#16] -Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] +(18) CometTakeOrderedAndProject +Input [3]: [brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt index 34630b3cb6..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS ext_price#7)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt index 428aac6c4d..dd7658cbac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,ext_price] + CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] CometProject [ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt index 801934db83..08e1bae2ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (19) -+- * HashAggregate (18) - +- * CometColumnarToRow (17) +* CometColumnarToRow (19) ++- CometTakeOrderedAndProject (18) + +- CometHashAggregate (17) +- CometExchange (16) +- CometHashAggregate (15) +- CometProject (14) @@ -97,17 +97,15 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [3]: [i_brand#11, i_brand_id#8, sum#12] Arguments: hashpartitioning(i_brand#11, i_brand_id#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 1] -Input [3]: [i_brand#11, i_brand_id#8, sum#12] - -(18) HashAggregate [codegen id : 1] +(17) CometHashAggregate Input [3]: [i_brand#11, i_brand_id#8, sum#12] Keys [2]: [i_brand#11, i_brand_id#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] -Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#11 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] -(19) TakeOrderedAndProject -Input [3]: [brand_id#14, brand#15, ext_price#16] -Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] +(18) CometTakeOrderedAndProject +Input [3]: [brand_id#13, brand#14, ext_price#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[ext_price#15 DESC NULLS LAST,brand_id#13 ASC NULLS FIRST], output=[brand_id#13,brand#14,ext_price#15]), [brand_id#13, brand#14, ext_price#15], 100, 0, [ext_price#15 DESC NULLS LAST, brand_id#13 ASC NULLS FIRST], [brand_id#13, brand#14, ext_price#15] + +(19) CometColumnarToRow [codegen id : 1] +Input [3]: [brand_id#13, brand#14, ext_price#15] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt index 34630b3cb6..a292badf5d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS ext_price#7)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -18,4 +18,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 16 out of 18 eligible operators (88%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 18 out of 18 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt index 428aac6c4d..dd7658cbac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [ext_price,brand_id,brand] - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [brand_id,brand,ext_price] + CometHashAggregate [sum] [brand_id,brand,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_brand,i_brand_id] #1 CometHashAggregate [ss_ext_sales_price] [i_brand,i_brand_id,sum] CometProject [ss_ext_sales_price,i_brand_id,i_brand] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt index e3e4ed07ce..16d7564ea5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/explain.txt @@ -1,69 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometColumnarExchange (62) - +- * HashAggregate (61) - +- Union (60) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (37) - : : +- CometBroadcastHashJoin (36) - : : :- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (59) - +- * CometColumnarToRow (58) - +- CometExchange (57) - +- CometHashAggregate (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (49) - : : +- CometBroadcastHashJoin (48) - : : :- CometFilter (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* CometColumnarToRow (62) ++- CometTakeOrderedAndProject (61) + +- CometHashAggregate (60) + +- CometExchange (59) + +- CometHashAggregate (58) + +- CometUnion (57) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometProject (53) + +- CometBroadcastHashJoin (52) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometFilter (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) + : : +- ReusedExchange (45) + : +- ReusedExchange (48) + +- ReusedExchange (51) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,211 +191,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_item_id#16, sum#17] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#16, sum#17] - -(29) HashAggregate [codegen id : 1] +(28) CometHashAggregate Input [2]: [i_item_id#16, sum#17] Keys [1]: [i_item_id#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [2]: [i_item_id#16 AS i_item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS total_sales#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : (isnotnull(cs_bill_addr_sk#21) AND isnotnull(cs_item_sk#22)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#26] +(30) CometFilter +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) -(33) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#24], [d_date_sk#26], Inner, BuildRight +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(34) CometProject -Input [5]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#26] -Arguments: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23], [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#27] +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(36) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Right output [1]: [ca_address_sk#27] -Arguments: [cs_bill_addr_sk#21], [ca_address_sk#27], Inner, BuildRight +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(37) CometProject -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, ca_address_sk#27] -Arguments: [cs_item_sk#22, cs_ext_sales_price#23], [cs_item_sk#22, cs_ext_sales_price#23] +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#28, i_item_id#29] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(39) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#22], [i_item_sk#28], Inner, BuildRight +(37) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#25, i_item_id#26] -(40) CometProject -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#23, i_item_id#29], [cs_ext_sales_price#23, i_item_id#29] +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight -(41) CometHashAggregate -Input [2]: [cs_ext_sales_price#23, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(42) CometExchange -Input [2]: [i_item_id#29, sum#30] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(43) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] +(41) CometExchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(44) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#31] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#31,17,2) AS total_sales#32] +(42) CometHashAggregate +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(44) CometFilter +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) -(47) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#38] +(45) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(48) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(49) CometProject -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38] -Arguments: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35], [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +(47) CometProject +Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] +Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -(50) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#39] +(48) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#34] -(51) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Right output [1]: [ca_address_sk#39] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#39], Inner, BuildRight +(49) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] +Right output [1]: [ca_address_sk#34] +Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight -(52) CometProject -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#39] -Arguments: [ws_item_sk#33, ws_ext_sales_price#35], [ws_item_sk#33, ws_ext_sales_price#35] +(50) CometProject +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] +Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#40, i_item_id#41] +(51) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#35, i_item_id#36] -(54) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Right output [2]: [i_item_sk#40, i_item_id#41] -Arguments: [ws_item_sk#33], [i_item_sk#40], Inner, BuildRight +(52) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] +Right output [2]: [i_item_sk#35, i_item_id#36] +Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight -(55) CometProject -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#40, i_item_id#41] -Arguments: [ws_ext_sales_price#35, i_item_id#41], [ws_ext_sales_price#35, i_item_id#41] +(53) CometProject +Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] +Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] + +(54) CometHashAggregate +Input [2]: [ws_ext_sales_price#30, i_item_id#36] +Keys [1]: [i_item_id#36] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] + +(55) CometExchange +Input [2]: [i_item_id#36, sum#37] +Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (56) CometHashAggregate -Input [2]: [ws_ext_sales_price#35, i_item_id#41] -Keys [1]: [i_item_id#41] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(57) CometExchange -Input [2]: [i_item_id#41, sum#42] -Arguments: hashpartitioning(i_item_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(58) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] - -(59) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] -Keys [1]: [i_item_id#41] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#41, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS total_sales#44] - -(60) Union - -(61) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#19, total_sales#20] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(total_sales#20)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [3]: [i_item_id#19, sum#47, isEmpty#48] - -(62) CometColumnarExchange -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(63) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] - -(64) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(total_sales#20)] -Aggregate Attributes [1]: [sum(total_sales#20)#49] -Results [2]: [i_item_id#19, sum(total_sales#20)#49 AS total_sales#50] - -(65) TakeOrderedAndProject -Input [2]: [i_item_id#19, total_sales#50] -Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_item_id#19, total_sales#50] +Input [2]: [i_item_id#36, sum#37] +Keys [1]: [i_item_id#36] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] + +(57) CometUnion +Child 0 Input [2]: [i_item_id#38, total_sales#39] +Child 1 Input [2]: [i_item_id#26, total_sales#40] +Child 2 Input [2]: [i_item_id#36, total_sales#41] + +(58) CometHashAggregate +Input [2]: [i_item_id#38, total_sales#39] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(total_sales#39)] + +(59) CometExchange +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(60) CometHashAggregate +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(total_sales#39)] + +(61) CometTakeOrderedAndProject +Input [2]: [i_item_id#38, total_sales#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] + +(62) CometColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#38, total_sales#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (67) ++- * CometColumnarToRow (66) + +- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,2), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(64) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) -(68) CometProject +(65) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(69) CometColumnarToRow [codegen id : 1] +(66) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(70) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt index 1b9273b540..36b9a8306d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/extended.txt @@ -1,103 +1,100 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS i_item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS total_sales#5)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#7))#8,17,2) AS total_sales#9)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#11))#12,17,2) AS total_sales#13)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 85 out of 96 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt index 2f7c5323ee..19eca782cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56.native_iceberg_compat/simplified.txt @@ -1,85 +1,74 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (5) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,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] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] + CometExchange [i_item_id] #1 + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] + CometUnion [i_item_id,total_sales] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,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] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_color] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt index e3e4ed07ce..16d7564ea5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt @@ -1,69 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometColumnarExchange (62) - +- * HashAggregate (61) - +- Union (60) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (37) - : : +- CometBroadcastHashJoin (36) - : : :- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (59) - +- * CometColumnarToRow (58) - +- CometExchange (57) - +- CometHashAggregate (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (49) - : : +- CometBroadcastHashJoin (48) - : : :- CometFilter (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* CometColumnarToRow (62) ++- CometTakeOrderedAndProject (61) + +- CometHashAggregate (60) + +- CometExchange (59) + +- CometHashAggregate (58) + +- CometUnion (57) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometProject (53) + +- CometBroadcastHashJoin (52) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometFilter (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) + : : +- ReusedExchange (45) + : +- ReusedExchange (48) + +- ReusedExchange (51) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,211 +191,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_item_id#16, sum#17] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#16, sum#17] - -(29) HashAggregate [codegen id : 1] +(28) CometHashAggregate Input [2]: [i_item_id#16, sum#17] Keys [1]: [i_item_id#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [2]: [i_item_id#16 AS i_item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS total_sales#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : (isnotnull(cs_bill_addr_sk#21) AND isnotnull(cs_item_sk#22)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#26] +(30) CometFilter +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) -(33) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#24], [d_date_sk#26], Inner, BuildRight +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(34) CometProject -Input [5]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#26] -Arguments: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23], [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#27] +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(36) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Right output [1]: [ca_address_sk#27] -Arguments: [cs_bill_addr_sk#21], [ca_address_sk#27], Inner, BuildRight +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(37) CometProject -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, ca_address_sk#27] -Arguments: [cs_item_sk#22, cs_ext_sales_price#23], [cs_item_sk#22, cs_ext_sales_price#23] +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#28, i_item_id#29] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(39) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#22], [i_item_sk#28], Inner, BuildRight +(37) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#25, i_item_id#26] -(40) CometProject -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#23, i_item_id#29], [cs_ext_sales_price#23, i_item_id#29] +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight -(41) CometHashAggregate -Input [2]: [cs_ext_sales_price#23, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(42) CometExchange -Input [2]: [i_item_id#29, sum#30] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(43) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] +(41) CometExchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(44) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#31] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#31,17,2) AS total_sales#32] +(42) CometHashAggregate +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(44) CometFilter +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) -(47) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#38] +(45) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(48) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(49) CometProject -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38] -Arguments: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35], [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +(47) CometProject +Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] +Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -(50) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#39] +(48) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#34] -(51) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Right output [1]: [ca_address_sk#39] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#39], Inner, BuildRight +(49) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] +Right output [1]: [ca_address_sk#34] +Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight -(52) CometProject -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#39] -Arguments: [ws_item_sk#33, ws_ext_sales_price#35], [ws_item_sk#33, ws_ext_sales_price#35] +(50) CometProject +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] +Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#40, i_item_id#41] +(51) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#35, i_item_id#36] -(54) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Right output [2]: [i_item_sk#40, i_item_id#41] -Arguments: [ws_item_sk#33], [i_item_sk#40], Inner, BuildRight +(52) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] +Right output [2]: [i_item_sk#35, i_item_id#36] +Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight -(55) CometProject -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#40, i_item_id#41] -Arguments: [ws_ext_sales_price#35, i_item_id#41], [ws_ext_sales_price#35, i_item_id#41] +(53) CometProject +Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] +Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] + +(54) CometHashAggregate +Input [2]: [ws_ext_sales_price#30, i_item_id#36] +Keys [1]: [i_item_id#36] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] + +(55) CometExchange +Input [2]: [i_item_id#36, sum#37] +Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (56) CometHashAggregate -Input [2]: [ws_ext_sales_price#35, i_item_id#41] -Keys [1]: [i_item_id#41] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(57) CometExchange -Input [2]: [i_item_id#41, sum#42] -Arguments: hashpartitioning(i_item_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(58) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] - -(59) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] -Keys [1]: [i_item_id#41] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#41, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS total_sales#44] - -(60) Union - -(61) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#19, total_sales#20] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(total_sales#20)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [3]: [i_item_id#19, sum#47, isEmpty#48] - -(62) CometColumnarExchange -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(63) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] - -(64) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(total_sales#20)] -Aggregate Attributes [1]: [sum(total_sales#20)#49] -Results [2]: [i_item_id#19, sum(total_sales#20)#49 AS total_sales#50] - -(65) TakeOrderedAndProject -Input [2]: [i_item_id#19, total_sales#50] -Arguments: 100, [total_sales#50 ASC NULLS FIRST], [i_item_id#19, total_sales#50] +Input [2]: [i_item_id#36, sum#37] +Keys [1]: [i_item_id#36] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] + +(57) CometUnion +Child 0 Input [2]: [i_item_id#38, total_sales#39] +Child 1 Input [2]: [i_item_id#26, total_sales#40] +Child 2 Input [2]: [i_item_id#36, total_sales#41] + +(58) CometHashAggregate +Input [2]: [i_item_id#38, total_sales#39] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(total_sales#39)] + +(59) CometExchange +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(60) CometHashAggregate +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(total_sales#39)] + +(61) CometTakeOrderedAndProject +Input [2]: [i_item_id#38, total_sales#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] + +(62) CometColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#38, total_sales#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (67) ++- * CometColumnarToRow (66) + +- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,2), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(64) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 2001)) AND (d_moy#8 = 2)) AND isnotnull(d_date_sk#6)) -(68) CometProject +(65) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(69) CometColumnarToRow [codegen id : 1] +(66) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(70) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt index 1b9273b540..36b9a8306d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/extended.txt @@ -1,103 +1,100 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS i_item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS total_sales#5)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#7))#8,17,2) AS total_sales#9)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#11))#12,17,2) AS total_sales#13)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 85 out of 96 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt index 2f7c5323ee..19eca782cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -1,85 +1,74 @@ -TakeOrderedAndProject [total_sales,i_item_id] - WholeStageCodegen (5) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,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] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_color] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] + CometExchange [i_item_id] #1 + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] + CometUnion [i_item_id,total_sales] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,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] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_color] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_color] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt index eb0ce14345..7fec2cf2e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] - -(22) HashAggregate [codegen id : 1] +(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))#17] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS _w0#19] -(23) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(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) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15, 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, 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] -(27) Filter [codegen id : 3] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(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)) -(28) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(29) Filter [codegen id : 10] -Input [9]: [i_category#5, i_brand#4, cc_name#15, 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) +(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] -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) HashAggregate [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] -Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] -Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] +(31) CometHashAggregate +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -(34) CometColumnarExchange -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -(37) Window -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +(36) Project [codegen id : 4] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] -(39) BroadcastExchange -Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(37) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 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#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(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#28] +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#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(44) Project [codegen id : 6] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(45) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] 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) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 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#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(47) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 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#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -(50) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt index ba90418971..de7d26cd06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt index dc90429f65..b854e818db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) + WholeStageCodegen (7) Project [i_category,i_brand,cc_name,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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index eb0ce14345..7fec2cf2e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] - -(22) HashAggregate [codegen id : 1] +(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))#17] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS _w0#19] -(23) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(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) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15, 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, 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] -(27) Filter [codegen id : 3] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(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)) -(28) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(29) Filter [codegen id : 10] -Input [9]: [i_category#5, i_brand#4, cc_name#15, 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) +(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] -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) HashAggregate [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] -Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] -Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] +(31) CometHashAggregate +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -(34) CometColumnarExchange -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -(37) Window -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +(36) Project [codegen id : 4] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] -(39) BroadcastExchange -Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(37) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 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#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(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#28] +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#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(44) Project [codegen id : 6] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(45) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] 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) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 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#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(47) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 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#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -(50) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt index ba90418971..de7d26cd06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index dc90429f65..b854e818db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) + WholeStageCodegen (7) Project [i_category,i_brand,cc_name,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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_ Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt index bcd506a593..3c368afc65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/explain.txt @@ -1,58 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (21) - : : +- * HashAggregate (20) - : : +- * CometColumnarToRow (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- 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 (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- ReusedExchange (27) - +- BroadcastExchange (51) - +- * Filter (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (40) - +- ReusedExchange (43) +* CometColumnarToRow (52) ++- CometTakeOrderedAndProject (51) + +- CometProject (50) + +- CometBroadcastHashJoin (49) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometFilter (20) + : : +- CometHashAggregate (19) + : : +- CometExchange (18) + : : +- CometHashAggregate (17) + : : +- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- 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 (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- ReusedExchange (11) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometHashAggregate (31) + : +- CometExchange (30) + : +- CometHashAggregate (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- CometBroadcastExchange (48) + +- CometFilter (47) + +- CometHashAggregate (46) + +- CometExchange (45) + +- CometHashAggregate (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometFilter (37) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (38) + +- ReusedExchange (41) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,7 +104,7 @@ ReadSchema: struct Input [2]: [d_date_sk#8, d_date#9] Condition : isnotnull(d_date_sk#8) -(11) ReusedExchange [Reuses operator id: 60] +(11) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date#10] (12) CometBroadcastHashJoin @@ -140,260 +138,246 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Input [2]: [i_item_id#7, sum#11] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(19) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#7, sum#11] - -(20) HashAggregate [codegen id : 3] +(19) CometHashAggregate Input [2]: [i_item_id#7, sum#11] Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [2]: [i_item_id#7 AS item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS ss_item_rev#14] -(21) Filter [codegen id : 3] -Input [2]: [item_id#13, ss_item_rev#14] -Condition : isnotnull(ss_item_rev#14) +(20) CometFilter +Input [2]: [item_id#12, ss_item_rev#13] +Condition : isnotnull(ss_item_rev#13) -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] 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#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_item_sk#15) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#19, i_item_id#20] - -(25) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Right output [2]: [i_item_sk#19, i_item_id#20] -Arguments: [cs_item_sk#15], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_sk#19, i_item_id#20] -Arguments: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20], [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] - -(27) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(28) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#17], [d_date_sk#21], Inner, BuildRight - -(29) CometProject -Input [4]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20, d_date_sk#21] -Arguments: [cs_ext_sales_price#16, i_item_id#20], [cs_ext_sales_price#16, i_item_id#20] - -(30) CometHashAggregate -Input [2]: [cs_ext_sales_price#16, i_item_id#20] -Keys [1]: [i_item_id#20] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#16))] - -(31) CometExchange -Input [2]: [i_item_id#20, sum#22] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#20, sum#22] - -(33) HashAggregate [codegen id : 1] -Input [2]: [i_item_id#20, sum#22] -Keys [1]: [i_item_id#20] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#16))#23] -Results [2]: [i_item_id#20 AS item_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#23,17,2) AS cs_item_rev#25] - -(34) Filter [codegen id : 1] -Input [2]: [item_id#24, cs_item_rev#25] -Condition : isnotnull(cs_item_rev#25) - -(35) BroadcastExchange -Input [2]: [item_id#24, cs_item_rev#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [item_id#13] -Right keys [1]: [item_id#24] -Join type: Inner -Join condition: ((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * cs_item_rev#25)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) - -(37) Project [codegen id : 3] -Output [3]: [item_id#13, ss_item_rev#14, cs_item_rev#25] -Input [4]: [item_id#13, ss_item_rev#14, item_id#24, cs_item_rev#25] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] +(22) CometFilter +Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#14) + +(23) ReusedExchange [Reuses operator id: 6] +Output [2]: [i_item_sk#18, i_item_id#19] + +(24) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight + +(25) CometProject +Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] +Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] + +(26) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#20] + +(27) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] +Right output [1]: [d_date_sk#20] +Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight + +(28) CometProject +Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] +Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] + +(29) CometHashAggregate +Input [2]: [cs_ext_sales_price#15, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] + +(30) CometExchange +Input [2]: [i_item_id#19, sum#21] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [2]: [i_item_id#19, sum#21] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] + +(32) CometFilter +Input [2]: [item_id#22, cs_item_rev#23] +Condition : isnotnull(cs_item_rev#23) + +(33) CometBroadcastExchange +Input [2]: [item_id#22, cs_item_rev#23] +Arguments: [item_id#22, cs_item_rev#23] + +(34) CometBroadcastHashJoin +Left output [2]: [item_id#12, ss_item_rev#13] +Right output [2]: [item_id#22, cs_item_rev#23] +Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight + +(35) CometProject +Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] +Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#28), dynamicpruningexpression(ws_sold_date_sk#28 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter -Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Condition : isnotnull(ws_item_sk#26) +(37) CometFilter +Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] +Condition : isnotnull(ws_item_sk#24) + +(38) ReusedExchange [Reuses operator id: 6] +Output [2]: [i_item_sk#28, i_item_id#29] -(40) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_item_id#31] +(39) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] +Right output [2]: [i_item_sk#28, i_item_id#29] +Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight -(41) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Right output [2]: [i_item_sk#30, i_item_id#31] -Arguments: [ws_item_sk#26], [i_item_sk#30], Inner, BuildRight +(40) CometProject +Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] +Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -(42) CometProject -Input [5]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_sk#30, i_item_id#31] -Arguments: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31], [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31] +(41) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#30] -(43) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#32] +(42) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] +Right output [1]: [d_date_sk#30] +Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31] -Right output [1]: [d_date_sk#32] -Arguments: [ws_sold_date_sk#28], [d_date_sk#32], Inner, BuildRight +(43) CometProject +Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] +Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] -(45) CometProject -Input [4]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31, d_date_sk#32] -Arguments: [ws_ext_sales_price#27, i_item_id#31], [ws_ext_sales_price#27, i_item_id#31] +(44) CometHashAggregate +Input [2]: [ws_ext_sales_price#25, i_item_id#29] +Keys [1]: [i_item_id#29] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] + +(45) CometExchange +Input [2]: [i_item_id#29, sum#31] +Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometHashAggregate -Input [2]: [ws_ext_sales_price#27, i_item_id#31] -Keys [1]: [i_item_id#31] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#27))] - -(47) CometExchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(48) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#31, sum#33] - -(49) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#27))#34] -Results [2]: [i_item_id#31 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#34,17,2) AS ws_item_rev#36] - -(50) Filter [codegen id : 2] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) - -(51) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(52) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [item_id#13] -Right keys [1]: [item_id#35] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#25))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) - -(53) Project [codegen id : 3] -Output [8]: [item_id#13, ss_item_rev#14, (((ss_item_rev#14 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#25, (((cs_item_rev#25 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#25, item_id#35, ws_item_rev#36] - -(54) TakeOrderedAndProject -Input [8]: [item_id#13, ss_item_rev#14, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +Input [2]: [i_item_id#29, sum#31] +Keys [1]: [i_item_id#29] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] -===== Subqueries ===== +(47) CometFilter +Input [2]: [item_id#32, ws_item_rev#33] +Condition : isnotnull(ws_item_rev#33) -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (60) - +- CometProject (59) - +- CometFilter (58) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (57) +(48) CometBroadcastExchange +Input [2]: [item_id#32, ws_item_rev#33] +Arguments: [item_id#32, ws_item_rev#33] +(49) CometBroadcastHashJoin +Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] +Right output [2]: [item_id#32, ws_item_rev#33] +Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(50) CometProject +Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] +Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] + +(51) CometTakeOrderedAndProject +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] + +(52) CometColumnarToRow [codegen id : 1] +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometFilter (54) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) + +- CometBroadcastExchange (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(56) CometFilter +(54) CometFilter Input [2]: [d_date_sk#8, d_date#9] Condition : isnotnull(d_date_sk#8) -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date#10, d_week_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#42)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#39)] ReadSchema: struct -(58) CometFilter -Input [2]: [d_date#10, d_week_seq#41] -Condition : (isnotnull(d_week_seq#41) AND (d_week_seq#41 = ReusedSubquery Subquery scalar-subquery#42, [id=#43])) +(56) CometFilter +Input [2]: [d_date#10, d_week_seq#38] +Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = ReusedSubquery Subquery scalar-subquery#39, [id=#40])) -(59) CometProject -Input [2]: [d_date#10, d_week_seq#41] +(57) CometProject +Input [2]: [d_date#10, d_week_seq#38] Arguments: [d_date#10], [d_date#10] -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [1]: [d_date#10] Arguments: [d_date#10] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [2]: [d_date_sk#8, d_date#9] Right output [1]: [d_date#10] Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight -(62) CometProject +(60) CometProject Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(63) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(64) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 58 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] -Subquery:3 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* CometColumnarToRow (68) -+- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +* CometColumnarToRow (66) ++- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#44, d_week_seq#45] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date#41, d_week_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(66) CometFilter -Input [2]: [d_date#44, d_week_seq#45] -Condition : (isnotnull(d_date#44) AND (d_date#44 = 2000-01-03)) +(64) CometFilter +Input [2]: [d_date#41, d_week_seq#42] +Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) -(67) CometProject -Input [2]: [d_date#44, d_week_seq#45] -Arguments: [d_week_seq#45], [d_week_seq#45] +(65) CometProject +Input [2]: [d_date#41, d_week_seq#42] +Arguments: [d_week_seq#42], [d_week_seq#42] -(68) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#45] +(66) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#42] -Subquery:4 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#28 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt index 4a144dd2d5..b4255cb7a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/extended.txt @@ -1,91 +1,89 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS ss_item_rev#5)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6 AS item_id#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#8))#9,17,2) AS cs_item_rev#10)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#11 AS item_id#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#13))#14,17,2) AS ws_item_rev#15)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- ReusedSubquery + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- 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 + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- ReusedSubquery + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -115,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 108 eligible operators (76%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt index 69bd5661b3..9d8bcdb40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58.native_iceberg_compat/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (3) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] + CometProject [item_id,ss_item_rev,cs_item_rev] + CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] + CometFilter [item_id,ss_item_rev] + CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #1 CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] CometProject [ss_ext_sales_price,i_item_id] @@ -47,39 +47,31 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometFilter [d_date_sk,d_date] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - 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] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - 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] + CometBroadcastExchange [item_id,cs_item_rev] #6 + CometFilter [item_id,cs_item_rev] + CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #7 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + 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] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [item_id,ws_item_rev] #8 + CometFilter [item_id,ws_item_rev] + CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + 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] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt index bcd506a593..3c368afc65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt @@ -1,58 +1,56 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (21) - : : +- * HashAggregate (20) - : : +- * CometColumnarToRow (19) - : : +- CometExchange (18) - : : +- CometHashAggregate (17) - : : +- CometProject (16) - : : +- CometBroadcastHashJoin (15) - : : :- 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 (14) - : : +- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- ReusedExchange (11) - : +- BroadcastExchange (35) - : +- * Filter (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- CometProject (29) - : +- CometBroadcastHashJoin (28) - : :- CometProject (26) - : : +- CometBroadcastHashJoin (25) - : : :- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (22) - : : +- ReusedExchange (24) - : +- ReusedExchange (27) - +- BroadcastExchange (51) - +- * Filter (50) - +- * HashAggregate (49) - +- * CometColumnarToRow (48) - +- CometExchange (47) - +- CometHashAggregate (46) - +- CometProject (45) - +- CometBroadcastHashJoin (44) - :- CometProject (42) - : +- CometBroadcastHashJoin (41) - : :- CometFilter (39) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (38) - : +- ReusedExchange (40) - +- ReusedExchange (43) +* CometColumnarToRow (52) ++- CometTakeOrderedAndProject (51) + +- CometProject (50) + +- CometBroadcastHashJoin (49) + :- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometFilter (20) + : : +- CometHashAggregate (19) + : : +- CometExchange (18) + : : +- CometHashAggregate (17) + : : +- CometProject (16) + : : +- CometBroadcastHashJoin (15) + : : :- 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 (14) + : : +- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- ReusedExchange (11) + : +- CometBroadcastExchange (33) + : +- CometFilter (32) + : +- CometHashAggregate (31) + : +- CometExchange (30) + : +- CometHashAggregate (29) + : +- CometProject (28) + : +- CometBroadcastHashJoin (27) + : :- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- CometFilter (22) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (21) + : : +- ReusedExchange (23) + : +- ReusedExchange (26) + +- CometBroadcastExchange (48) + +- CometFilter (47) + +- CometHashAggregate (46) + +- CometExchange (45) + +- CometHashAggregate (44) + +- CometProject (43) + +- CometBroadcastHashJoin (42) + :- CometProject (40) + : +- CometBroadcastHashJoin (39) + : :- CometFilter (37) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (36) + : +- ReusedExchange (38) + +- ReusedExchange (41) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,7 +104,7 @@ ReadSchema: struct Input [2]: [d_date_sk#8, d_date#9] Condition : isnotnull(d_date_sk#8) -(11) ReusedExchange [Reuses operator id: 60] +(11) ReusedExchange [Reuses operator id: 58] Output [1]: [d_date#10] (12) CometBroadcastHashJoin @@ -140,260 +138,246 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] Input [2]: [i_item_id#7, sum#11] Arguments: hashpartitioning(i_item_id#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(19) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#7, sum#11] - -(20) HashAggregate [codegen id : 3] +(19) CometHashAggregate Input [2]: [i_item_id#7, sum#11] Keys [1]: [i_item_id#7] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] -Results [2]: [i_item_id#7 AS item_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS ss_item_rev#14] -(21) Filter [codegen id : 3] -Input [2]: [item_id#13, ss_item_rev#14] -Condition : isnotnull(ss_item_rev#14) +(20) CometFilter +Input [2]: [item_id#12, ss_item_rev#13] +Condition : isnotnull(ss_item_rev#13) -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] 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#16), dynamicpruningexpression(cs_sold_date_sk#16 IN dynamicpruning#17)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Condition : isnotnull(cs_item_sk#15) - -(24) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#19, i_item_id#20] - -(25) CometBroadcastHashJoin -Left output [3]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] -Right output [2]: [i_item_sk#19, i_item_id#20] -Arguments: [cs_item_sk#15], [i_item_sk#19], Inner, BuildRight - -(26) CometProject -Input [5]: [cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_sk#19, i_item_id#20] -Arguments: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20], [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] - -(27) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(28) CometBroadcastHashJoin -Left output [3]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20] -Right output [1]: [d_date_sk#21] -Arguments: [cs_sold_date_sk#17], [d_date_sk#21], Inner, BuildRight - -(29) CometProject -Input [4]: [cs_ext_sales_price#16, cs_sold_date_sk#17, i_item_id#20, d_date_sk#21] -Arguments: [cs_ext_sales_price#16, i_item_id#20], [cs_ext_sales_price#16, i_item_id#20] - -(30) CometHashAggregate -Input [2]: [cs_ext_sales_price#16, i_item_id#20] -Keys [1]: [i_item_id#20] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#16))] - -(31) CometExchange -Input [2]: [i_item_id#20, sum#22] -Arguments: hashpartitioning(i_item_id#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(32) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#20, sum#22] - -(33) HashAggregate [codegen id : 1] -Input [2]: [i_item_id#20, sum#22] -Keys [1]: [i_item_id#20] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#16))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#16))#23] -Results [2]: [i_item_id#20 AS item_id#24, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#23,17,2) AS cs_item_rev#25] - -(34) Filter [codegen id : 1] -Input [2]: [item_id#24, cs_item_rev#25] -Condition : isnotnull(cs_item_rev#25) - -(35) BroadcastExchange -Input [2]: [item_id#24, cs_item_rev#25] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(36) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [item_id#13] -Right keys [1]: [item_id#24] -Join type: Inner -Join condition: ((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * cs_item_rev#25)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) - -(37) Project [codegen id : 3] -Output [3]: [item_id#13, ss_item_rev#14, cs_item_rev#25] -Input [4]: [item_id#13, ss_item_rev#14, item_id#24, cs_item_rev#25] - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] +(22) CometFilter +Input [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Condition : isnotnull(cs_item_sk#14) + +(23) ReusedExchange [Reuses operator id: 6] +Output [2]: [i_item_sk#18, i_item_id#19] + +(24) CometBroadcastHashJoin +Left output [3]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16] +Right output [2]: [i_item_sk#18, i_item_id#19] +Arguments: [cs_item_sk#14], [i_item_sk#18], Inner, BuildRight + +(25) CometProject +Input [5]: [cs_item_sk#14, cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_sk#18, i_item_id#19] +Arguments: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19], [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] + +(26) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#20] + +(27) CometBroadcastHashJoin +Left output [3]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19] +Right output [1]: [d_date_sk#20] +Arguments: [cs_sold_date_sk#16], [d_date_sk#20], Inner, BuildRight + +(28) CometProject +Input [4]: [cs_ext_sales_price#15, cs_sold_date_sk#16, i_item_id#19, d_date_sk#20] +Arguments: [cs_ext_sales_price#15, i_item_id#19], [cs_ext_sales_price#15, i_item_id#19] + +(29) CometHashAggregate +Input [2]: [cs_ext_sales_price#15, i_item_id#19] +Keys [1]: [i_item_id#19] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#15))] + +(30) CometExchange +Input [2]: [i_item_id#19, sum#21] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(31) CometHashAggregate +Input [2]: [i_item_id#19, sum#21] +Keys [1]: [i_item_id#19] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#15))] + +(32) CometFilter +Input [2]: [item_id#22, cs_item_rev#23] +Condition : isnotnull(cs_item_rev#23) + +(33) CometBroadcastExchange +Input [2]: [item_id#22, cs_item_rev#23] +Arguments: [item_id#22, cs_item_rev#23] + +(34) CometBroadcastHashJoin +Left output [2]: [item_id#12, ss_item_rev#13] +Right output [2]: [item_id#22, cs_item_rev#23] +Arguments: [item_id#12], [item_id#22], Inner, ((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#13))), BuildRight + +(35) CometProject +Input [4]: [item_id#12, ss_item_rev#13, item_id#22, cs_item_rev#23] +Arguments: [item_id#12, ss_item_rev#13, cs_item_rev#23], [item_id#12, ss_item_rev#13, cs_item_rev#23] + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#28), dynamicpruningexpression(ws_sold_date_sk#28 IN dynamicpruning#29)] +PartitionFilters: [isnotnull(ws_sold_date_sk#26), dynamicpruningexpression(ws_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(39) CometFilter -Input [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Condition : isnotnull(ws_item_sk#26) +(37) CometFilter +Input [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] +Condition : isnotnull(ws_item_sk#24) + +(38) ReusedExchange [Reuses operator id: 6] +Output [2]: [i_item_sk#28, i_item_id#29] -(40) ReusedExchange [Reuses operator id: 6] -Output [2]: [i_item_sk#30, i_item_id#31] +(39) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26] +Right output [2]: [i_item_sk#28, i_item_id#29] +Arguments: [ws_item_sk#24], [i_item_sk#28], Inner, BuildRight -(41) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28] -Right output [2]: [i_item_sk#30, i_item_id#31] -Arguments: [ws_item_sk#26], [i_item_sk#30], Inner, BuildRight +(40) CometProject +Input [5]: [ws_item_sk#24, ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_sk#28, i_item_id#29] +Arguments: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29], [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] -(42) CometProject -Input [5]: [ws_item_sk#26, ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_sk#30, i_item_id#31] -Arguments: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31], [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31] +(41) ReusedExchange [Reuses operator id: 14] +Output [1]: [d_date_sk#30] -(43) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#32] +(42) CometBroadcastHashJoin +Left output [3]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29] +Right output [1]: [d_date_sk#30] +Arguments: [ws_sold_date_sk#26], [d_date_sk#30], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [3]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31] -Right output [1]: [d_date_sk#32] -Arguments: [ws_sold_date_sk#28], [d_date_sk#32], Inner, BuildRight +(43) CometProject +Input [4]: [ws_ext_sales_price#25, ws_sold_date_sk#26, i_item_id#29, d_date_sk#30] +Arguments: [ws_ext_sales_price#25, i_item_id#29], [ws_ext_sales_price#25, i_item_id#29] -(45) CometProject -Input [4]: [ws_ext_sales_price#27, ws_sold_date_sk#28, i_item_id#31, d_date_sk#32] -Arguments: [ws_ext_sales_price#27, i_item_id#31], [ws_ext_sales_price#27, i_item_id#31] +(44) CometHashAggregate +Input [2]: [ws_ext_sales_price#25, i_item_id#29] +Keys [1]: [i_item_id#29] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#25))] + +(45) CometExchange +Input [2]: [i_item_id#29, sum#31] +Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometHashAggregate -Input [2]: [ws_ext_sales_price#27, i_item_id#31] -Keys [1]: [i_item_id#31] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#27))] - -(47) CometExchange -Input [2]: [i_item_id#31, sum#33] -Arguments: hashpartitioning(i_item_id#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(48) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#31, sum#33] - -(49) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#31, sum#33] -Keys [1]: [i_item_id#31] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#27))#34] -Results [2]: [i_item_id#31 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#34,17,2) AS ws_item_rev#36] - -(50) Filter [codegen id : 2] -Input [2]: [item_id#35, ws_item_rev#36] -Condition : isnotnull(ws_item_rev#36) - -(51) BroadcastExchange -Input [2]: [item_id#35, ws_item_rev#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(52) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [item_id#13] -Right keys [1]: [item_id#35] -Join type: Inner -Join condition: ((((((((cast(ss_item_rev#14 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#14 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#25 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#14))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#14))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#25))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#25))) - -(53) Project [codegen id : 3] -Output [8]: [item_id#13, ss_item_rev#14, (((ss_item_rev#14 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#25, (((cs_item_rev#25 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#14 + cs_item_rev#25) + ws_item_rev#36) / 3) AS average#40] -Input [5]: [item_id#13, ss_item_rev#14, cs_item_rev#25, item_id#35, ws_item_rev#36] - -(54) TakeOrderedAndProject -Input [8]: [item_id#13, ss_item_rev#14, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] -Arguments: 100, [item_id#13 ASC NULLS FIRST, ss_item_rev#14 ASC NULLS FIRST], [item_id#13, ss_item_rev#14, ss_dev#37, cs_item_rev#25, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +Input [2]: [i_item_id#29, sum#31] +Keys [1]: [i_item_id#29] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#25))] -===== Subqueries ===== +(47) CometFilter +Input [2]: [item_id#32, ws_item_rev#33] +Condition : isnotnull(ws_item_rev#33) -Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (64) -+- * CometColumnarToRow (63) - +- CometProject (62) - +- CometBroadcastHashJoin (61) - :- CometFilter (56) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) - +- CometBroadcastExchange (60) - +- CometProject (59) - +- CometFilter (58) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (57) +(48) CometBroadcastExchange +Input [2]: [item_id#32, ws_item_rev#33] +Arguments: [item_id#32, ws_item_rev#33] +(49) CometBroadcastHashJoin +Left output [3]: [item_id#12, ss_item_rev#13, cs_item_rev#23] +Right output [2]: [item_id#32, ws_item_rev#33] +Arguments: [item_id#12], [item_id#32], Inner, ((((((((cast(ss_item_rev#13 as decimal(19,3)) >= (0.9 * ws_item_rev#33)) AND (cast(ss_item_rev#13 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#33))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#33))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * ss_item_rev#13))) AND (cast(ws_item_rev#33 as decimal(19,3)) >= (0.9 * cs_item_rev#23))) AND (cast(ws_item_rev#33 as decimal(20,3)) <= (1.1 * cs_item_rev#23))), BuildRight -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(50) CometProject +Input [5]: [item_id#12, ss_item_rev#13, cs_item_rev#23, item_id#32, ws_item_rev#33] +Arguments: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], [item_id#12, ss_item_rev#13, (((ss_item_rev#13 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ss_dev#34, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS cs_dev#35, ws_item_rev#33, (((ws_item_rev#33 / ((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33)) / 3) * 100) AS ws_dev#36, (((ss_item_rev#13 + cs_item_rev#23) + ws_item_rev#33) / 3) AS average#37] + +(51) CometTakeOrderedAndProject +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_id#12 ASC NULLS FIRST,ss_item_rev#13 ASC NULLS FIRST], output=[item_id#12,ss_item_rev#13,ss_dev#34,cs_item_rev#23,cs_dev#35,ws_item_rev#33,ws_dev#36,average#37]), [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37], 100, 0, [item_id#12 ASC NULLS FIRST, ss_item_rev#13 ASC NULLS FIRST], [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] + +(52) CometColumnarToRow [codegen id : 1] +Input [8]: [item_id#12, ss_item_rev#13, ss_dev#34, cs_item_rev#23, cs_dev#35, ws_item_rev#33, ws_dev#36, average#37] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometBroadcastHashJoin (59) + :- CometFilter (54) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (53) + +- CometBroadcastExchange (58) + +- CometProject (57) + +- CometFilter (56) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) + + +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_date#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date_sk)] ReadSchema: struct -(56) CometFilter +(54) CometFilter Input [2]: [d_date_sk#8, d_date#9] Condition : isnotnull(d_date_sk#8) -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#10, d_week_seq#41] +(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date#10, d_week_seq#38] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#42)] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#39)] ReadSchema: struct -(58) CometFilter -Input [2]: [d_date#10, d_week_seq#41] -Condition : (isnotnull(d_week_seq#41) AND (d_week_seq#41 = ReusedSubquery Subquery scalar-subquery#42, [id=#43])) +(56) CometFilter +Input [2]: [d_date#10, d_week_seq#38] +Condition : (isnotnull(d_week_seq#38) AND (d_week_seq#38 = ReusedSubquery Subquery scalar-subquery#39, [id=#40])) -(59) CometProject -Input [2]: [d_date#10, d_week_seq#41] +(57) CometProject +Input [2]: [d_date#10, d_week_seq#38] Arguments: [d_date#10], [d_date#10] -(60) CometBroadcastExchange +(58) CometBroadcastExchange Input [1]: [d_date#10] Arguments: [d_date#10] -(61) CometBroadcastHashJoin +(59) CometBroadcastHashJoin Left output [2]: [d_date_sk#8, d_date#9] Right output [1]: [d_date#10] Arguments: [d_date#9], [d_date#10], LeftSemi, BuildRight -(62) CometProject +(60) CometProject Input [2]: [d_date_sk#8, d_date#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(63) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(64) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 58 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#43] +Subquery:2 Hosting operator id = 56 Hosting Expression = ReusedSubquery Subquery scalar-subquery#39, [id=#40] -Subquery:3 Hosting operator id = 57 Hosting Expression = Subquery scalar-subquery#42, [id=#43] -* CometColumnarToRow (68) -+- CometProject (67) - +- CometFilter (66) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (65) +Subquery:3 Hosting operator id = 55 Hosting Expression = Subquery scalar-subquery#39, [id=#40] +* CometColumnarToRow (66) ++- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(65) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date#44, d_week_seq#45] +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date#41, d_week_seq#42] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] ReadSchema: struct -(66) CometFilter -Input [2]: [d_date#44, d_week_seq#45] -Condition : (isnotnull(d_date#44) AND (d_date#44 = 2000-01-03)) +(64) CometFilter +Input [2]: [d_date#41, d_week_seq#42] +Condition : (isnotnull(d_date#41) AND (d_date#41 = 2000-01-03)) -(67) CometProject -Input [2]: [d_date#44, d_week_seq#45] -Arguments: [d_week_seq#45], [d_week_seq#45] +(65) CometProject +Input [2]: [d_date#41, d_week_seq#42] +Arguments: [d_week_seq#42], [d_week_seq#42] -(68) CometColumnarToRow [codegen id : 1] -Input [1]: [d_week_seq#45] +(66) CometColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#42] -Subquery:4 Hosting operator id = 22 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#4 +Subquery:4 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#16 IN dynamicpruning#4 -Subquery:5 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#28 IN dynamicpruning#4 +Subquery:5 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt index 4a144dd2d5..b4255cb7a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/extended.txt @@ -1,91 +1,89 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Filter - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS ss_item_rev#5)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : : +- ReusedSubquery - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : : +- Subquery - : : : : +- 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 - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : : +- ReusedSubquery - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- Subquery - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6 AS item_id#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#8))#9,17,2) AS cs_item_rev#10)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : : +- ReusedSubquery - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Subquery - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#11 AS item_id#12, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#13))#14,17,2) AS ws_item_rev#15)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : : +- ReusedSubquery + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : : +- Subquery + : : : : +- 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 + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : : +- ReusedSubquery + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- Subquery + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : : +- ReusedSubquery + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- Subquery + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -115,4 +113,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 108 eligible operators (76%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 108 eligible operators (88%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt index 69bd5661b3..9d8bcdb40a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -1,13 +1,13 @@ -TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] - WholeStageCodegen (3) - Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] - Project [item_id,ss_item_rev,cs_item_rev] - BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] - Filter [ss_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + CometBroadcastHashJoin [item_id,ss_item_rev,cs_item_rev,item_id,ws_item_rev] + CometProject [item_id,ss_item_rev,cs_item_rev] + CometBroadcastHashJoin [item_id,ss_item_rev,item_id,cs_item_rev] + CometFilter [item_id,ss_item_rev] + CometHashAggregate [sum] [item_id,ss_item_rev,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] CometExchange [i_item_id] #1 CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] CometProject [ss_ext_sales_price,i_item_id] @@ -47,39 +47,31 @@ TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev CometFilter [d_date_sk,d_date] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] ReusedExchange [d_date] #3 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - Filter [cs_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #7 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] - 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] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [i_item_sk,i_item_id] #4 - ReusedExchange [d_date_sk] #5 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - Filter [ws_item_rev] - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] - 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] + CometBroadcastExchange [item_id,cs_item_rev] #6 + CometFilter [item_id,cs_item_rev] + CometHashAggregate [sum] [item_id,cs_item_rev,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #7 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id] + 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] ReusedSubquery [d_date_sk] #1 ReusedExchange [i_item_sk,i_item_id] #4 ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [item_id,ws_item_rev] #8 + CometFilter [item_id,ws_item_rev] + CometHashAggregate [sum] [item_id,ws_item_rev,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id] + 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] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [i_item_sk,i_item_id] #4 + ReusedExchange [d_date_sk] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt index a9d8494074..8fb48d458e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/explain.txt @@ -1,51 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (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 (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - +- BroadcastExchange (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- ReusedExchange (27) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (30) - +- BroadcastExchange (41) - +- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (37) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (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) + : : +- CometBroadcastExchange (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) + : +- CometBroadcastExchange (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) + +- CometBroadcastExchange (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometHashAggregate (25) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -97,172 +92,148 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) T Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometColumnarToRow [codegen id : 6] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] - -(12) HashAggregate [codegen id : 6] +(11) CometHashAggregate Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))#21] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END))#15,17,2) AS sun_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END))#16,17,2) AS mon_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END))#17,17,2) AS tue_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END))#18,17,2) AS wed_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END))#19,17,2) AS thu_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END))#20,17,2) AS fri_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))#21,17,2) AS sat_sales#28] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Condition : (isnotnull(s_store_sk#29) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#30, 16)))) - -(15) CometProject -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Arguments: [s_store_sk#29, s_store_id#32, s_store_name#31], [s_store_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#30, 16)) AS s_store_id#32, s_store_name#31] +(13) CometFilter +Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] +Condition : (isnotnull(s_store_sk#15) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)))) -(16) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#29, s_store_id#32, s_store_name#31] +(14) CometProject +Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] +Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)) AS s_store_id#18, s_store_name#17] -(17) BroadcastExchange -Input [3]: [s_store_sk#29, s_store_id#32, s_store_name#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(15) CometBroadcastExchange +Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#29] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] +Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight -(19) Project [codegen id : 6] -Output [10]: [d_week_seq#5, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_id#32, s_store_name#31] -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_sk#29, s_store_id#32, s_store_name#31] +(17) CometProject +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#33, d_week_seq#34] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#26, d_week_seq#27] 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_week_seq)] ReadSchema: struct -(21) CometFilter -Input [2]: [d_month_seq#33, d_week_seq#34] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1212)) AND (d_month_seq#33 <= 1223)) AND isnotnull(d_week_seq#34)) +(19) CometFilter +Input [2]: [d_month_seq#26, d_week_seq#27] +Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) -(22) CometProject -Input [2]: [d_month_seq#33, d_week_seq#34] -Arguments: [d_week_seq#34], [d_week_seq#34] +(20) CometProject +Input [2]: [d_month_seq#26, d_week_seq#27] +Arguments: [d_week_seq#27], [d_week_seq#27] -(23) CometColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#34] +(21) CometBroadcastExchange +Input [1]: [d_week_seq#27] +Arguments: [d_week_seq#27] -(24) BroadcastExchange -Input [1]: [d_week_seq#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] +Right output [1]: [d_week_seq#27] +Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#34] -Join type: Inner -Join condition: None +(23) CometProject +Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] +Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] -(26) Project [codegen id : 6] -Output [10]: [s_store_name#31 AS s_store_name1#35, d_week_seq#5 AS d_week_seq1#36, s_store_id#32 AS s_store_id1#37, sun_sales#22 AS sun_sales1#38, mon_sales#23 AS mon_sales1#39, tue_sales#24 AS tue_sales1#40, wed_sales#25 AS wed_sales1#41, thu_sales#26 AS thu_sales1#42, fri_sales#27 AS fri_sales1#43, sat_sales#28 AS sat_sales1#44] -Input [11]: [d_week_seq#5, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_id#32, s_store_name#31, d_week_seq#34] +(24) ReusedExchange [Reuses operator id: 10] +Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -(27) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] +(25) CometHashAggregate +Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] +Keys [2]: [d_week_seq#38, ss_store_sk#39] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#47 = Sunday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Monday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Tuesday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Wednesday) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Thursday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Friday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Saturday ) THEN ss_sales_price#48 END))] -(28) CometColumnarToRow [codegen id : 5] -Input [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] - -(29) HashAggregate [codegen id : 5] -Input [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] -Keys [2]: [d_week_seq#45, ss_store_sk#46] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))#21] -Results [9]: [d_week_seq#45, ss_store_sk#46, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END))#15,17,2) AS sun_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END))#16,17,2) AS mon_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END))#17,17,2) AS tue_sales#58, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END))#18,17,2) AS wed_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END))#19,17,2) AS thu_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END))#20,17,2) AS fri_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))#21,17,2) AS sat_sales#62] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#63, s_store_id#64] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#49, s_store_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [s_store_sk#63, s_store_id#64] -Condition : (isnotnull(s_store_sk#63) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#64, 16)))) - -(32) CometProject -Input [2]: [s_store_sk#63, s_store_id#64] -Arguments: [s_store_sk#63, s_store_id#65], [s_store_sk#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#64, 16)) AS s_store_id#65] +(27) CometFilter +Input [2]: [s_store_sk#49, s_store_id#50] +Condition : (isnotnull(s_store_sk#49) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)))) -(33) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#63, s_store_id#65] +(28) CometProject +Input [2]: [s_store_sk#49, s_store_id#50] +Arguments: [s_store_sk#49, s_store_id#51], [s_store_sk#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)) AS s_store_id#51] -(34) BroadcastExchange -Input [2]: [s_store_sk#63, s_store_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(29) CometBroadcastExchange +Input [2]: [s_store_sk#49, s_store_id#51] +Arguments: [s_store_sk#49, s_store_id#51] -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#46] -Right keys [1]: [s_store_sk#63] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58] +Right output [2]: [s_store_sk#49, s_store_id#51] +Arguments: [ss_store_sk#39], [s_store_sk#49], Inner, BuildRight -(36) Project [codegen id : 5] -Output [9]: [d_week_seq#45, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_id#65] -Input [11]: [d_week_seq#45, ss_store_sk#46, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_sk#63, s_store_id#65] +(31) CometProject +Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_sk#49, s_store_id#51] +Arguments: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51], [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#66, d_week_seq#67] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#59, d_week_seq#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(38) CometFilter -Input [2]: [d_month_seq#66, d_week_seq#67] -Condition : (((isnotnull(d_month_seq#66) AND (d_month_seq#66 >= 1224)) AND (d_month_seq#66 <= 1235)) AND isnotnull(d_week_seq#67)) +(33) CometFilter +Input [2]: [d_month_seq#59, d_week_seq#60] +Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) -(39) CometProject -Input [2]: [d_month_seq#66, d_week_seq#67] -Arguments: [d_week_seq#67], [d_week_seq#67] +(34) CometProject +Input [2]: [d_month_seq#59, d_week_seq#60] +Arguments: [d_week_seq#60], [d_week_seq#60] -(40) CometColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#67] +(35) CometBroadcastExchange +Input [1]: [d_week_seq#60] +Arguments: [d_week_seq#60] -(41) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [9]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] +Right output [1]: [d_week_seq#60] +Arguments: [d_week_seq#38], [d_week_seq#60], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_week_seq#45] -Right keys [1]: [d_week_seq#67] -Join type: Inner -Join condition: None +(37) CometProject +Input [10]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51, d_week_seq#60] +Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69], [d_week_seq#38 AS d_week_seq2#61, s_store_id#51 AS s_store_id2#62, sun_sales#52 AS sun_sales2#63, mon_sales#53 AS mon_sales2#64, tue_sales#54 AS tue_sales2#65, wed_sales#55 AS wed_sales2#66, thu_sales#56 AS thu_sales2#67, fri_sales#57 AS fri_sales2#68, sat_sales#58 AS sat_sales2#69] -(43) Project [codegen id : 5] -Output [9]: [d_week_seq#45 AS d_week_seq2#68, s_store_id#65 AS s_store_id2#69, sun_sales#56 AS sun_sales2#70, mon_sales#57 AS mon_sales2#71, tue_sales#58 AS tue_sales2#72, wed_sales#59 AS wed_sales2#73, thu_sales#60 AS thu_sales2#74, fri_sales#61 AS fri_sales2#75, sat_sales#62 AS sat_sales2#76] -Input [10]: [d_week_seq#45, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_id#65, d_week_seq#67] +(38) CometBroadcastExchange +Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -(44) BroadcastExchange -Input [9]: [d_week_seq2#68, s_store_id2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] +Right output [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#62, (d_week_seq2#61 - 52)], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [s_store_id1#37, d_week_seq1#36] -Right keys [2]: [s_store_id2#69, (d_week_seq2#68 - 52)] -Join type: Inner -Join condition: None +(40) CometProject +Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#32 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#33 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#34 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#35 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#36 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#37 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] -(46) Project [codegen id : 6] -Output [10]: [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1#38 / sun_sales2#70) AS (sun_sales1 / sun_sales2)#77, (mon_sales1#39 / mon_sales2#71) AS (mon_sales1 / mon_sales2)#78, (tue_sales1#40 / tue_sales2#72) AS (tue_sales1 / tue_sales2)#79, (wed_sales1#41 / wed_sales2#73) AS (wed_sales1 / wed_sales2)#80, (thu_sales1#42 / thu_sales2#74) AS (thu_sales1 / thu_sales2)#81, (fri_sales1#43 / fri_sales2#75) AS (fri_sales1 / fri_sales2)#82, (sat_sales1#44 / sat_sales2#76) AS (sat_sales1 / sat_sales2)#83] -Input [19]: [s_store_name1#35, d_week_seq1#36, s_store_id1#37, sun_sales1#38, mon_sales1#39, tue_sales1#40, wed_sales1#41, thu_sales1#42, fri_sales1#43, sat_sales1#44, d_week_seq2#68, s_store_id2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] +(41) CometTakeOrderedAndProject +Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -(47) TakeOrderedAndProject -Input [10]: [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] -Arguments: 100, [s_store_name1#35 ASC NULLS FIRST, s_store_id1#37 ASC NULLS FIRST, d_week_seq1#36 ASC NULLS FIRST], [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +(42) CometColumnarToRow [codegen id : 1] +Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt index 333907763b..3d50a2131e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/extended.txt @@ -1,58 +1,53 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#1, ss_store_sk#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#4 END))#5,17,2) AS sun_sales#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#4 END))#7,17,2) AS mon_sales#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#4 END))#9,17,2) AS tue_sales#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#4 END))#11,17,2) AS wed_sales#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#4 END))#13,17,2) AS thu_sales#14, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#4 END))#15,17,2) AS fri_sales#16, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#4 END))#17,17,2) AS sat_sales#18)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- 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 - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#19, ss_store_sk#20, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Sunday ) THEN ss_sales_price#22 END))#5,17,2) AS sun_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Monday ) THEN ss_sales_price#22 END))#7,17,2) AS mon_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Tuesday ) THEN ss_sales_price#22 END))#9,17,2) AS tue_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Wednesday) THEN ss_sales_price#22 END))#11,17,2) AS wed_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Thursday ) THEN ss_sales_price#22 END))#13,17,2) AS thu_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Friday ) THEN ss_sales_price#22 END))#15,17,2) AS fri_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Saturday ) THEN ss_sales_price#22 END))#17,17,2) AS sat_sales#29)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 50 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt index 867d0d888e..bca9c6463d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59.native_iceberg_compat/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - WholeStageCodegen (6) - Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] - Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] CometExchange [d_week_seq,ss_store_sk] #1 CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] @@ -19,46 +19,26 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - 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 #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + 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] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt index a9d8494074..8fb48d458e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt @@ -1,51 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (47) -+- * Project (46) - +- * BroadcastHashJoin Inner BuildRight (45) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (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 (17) - : : +- * CometColumnarToRow (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (20) - +- BroadcastExchange (44) - +- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- ReusedExchange (27) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (30) - +- BroadcastExchange (41) - +- * CometColumnarToRow (40) - +- CometProject (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (37) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (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) + : : +- CometBroadcastExchange (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (12) + : +- CometBroadcastExchange (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (18) + +- CometBroadcastExchange (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometHashAggregate (25) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (26) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -97,172 +92,148 @@ Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) T Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(11) CometColumnarToRow [codegen id : 6] -Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] - -(12) HashAggregate [codegen id : 6] +(11) CometHashAggregate Input [9]: [d_week_seq#5, ss_store_sk#1, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13, sum#14] Keys [2]: [d_week_seq#5, ss_store_sk#1] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))#21] -Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Sunday ) THEN ss_sales_price#2 END))#15,17,2) AS sun_sales#22, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Monday ) THEN ss_sales_price#2 END))#16,17,2) AS mon_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Tuesday ) THEN ss_sales_price#2 END))#17,17,2) AS tue_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Wednesday) THEN ss_sales_price#2 END))#18,17,2) AS wed_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Thursday ) THEN ss_sales_price#2 END))#19,17,2) AS thu_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Friday ) THEN ss_sales_price#2 END))#20,17,2) AS fri_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#7 = Saturday ) THEN ss_sales_price#2 END))#21,17,2) AS sat_sales#28] -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] +(12) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Condition : (isnotnull(s_store_sk#29) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#30, 16)))) - -(15) CometProject -Input [3]: [s_store_sk#29, s_store_id#30, s_store_name#31] -Arguments: [s_store_sk#29, s_store_id#32, s_store_name#31], [s_store_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#30, 16)) AS s_store_id#32, s_store_name#31] +(13) CometFilter +Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] +Condition : (isnotnull(s_store_sk#15) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)))) -(16) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#29, s_store_id#32, s_store_name#31] +(14) CometProject +Input [3]: [s_store_sk#15, s_store_id#16, s_store_name#17] +Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17], [s_store_sk#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#16, 16)) AS s_store_id#18, s_store_name#17] -(17) BroadcastExchange -Input [3]: [s_store_sk#29, s_store_id#32, s_store_name#31] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(15) CometBroadcastExchange +Input [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [s_store_sk#15, s_store_id#18, s_store_name#17] -(18) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#29] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [9]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25] +Right output [3]: [s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [ss_store_sk#1], [s_store_sk#15], Inner, BuildRight -(19) Project [codegen id : 6] -Output [10]: [d_week_seq#5, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_id#32, s_store_name#31] -Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_sk#29, s_store_id#32, s_store_name#31] +(17) CometProject +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_sk#15, s_store_id#18, s_store_name#17] +Arguments: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17], [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#33, d_week_seq#34] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#26, d_week_seq#27] 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_week_seq)] ReadSchema: struct -(21) CometFilter -Input [2]: [d_month_seq#33, d_week_seq#34] -Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1212)) AND (d_month_seq#33 <= 1223)) AND isnotnull(d_week_seq#34)) +(19) CometFilter +Input [2]: [d_month_seq#26, d_week_seq#27] +Condition : (((isnotnull(d_month_seq#26) AND (d_month_seq#26 >= 1212)) AND (d_month_seq#26 <= 1223)) AND isnotnull(d_week_seq#27)) -(22) CometProject -Input [2]: [d_month_seq#33, d_week_seq#34] -Arguments: [d_week_seq#34], [d_week_seq#34] +(20) CometProject +Input [2]: [d_month_seq#26, d_week_seq#27] +Arguments: [d_week_seq#27], [d_week_seq#27] -(23) CometColumnarToRow [codegen id : 2] -Input [1]: [d_week_seq#34] +(21) CometBroadcastExchange +Input [1]: [d_week_seq#27] +Arguments: [d_week_seq#27] -(24) BroadcastExchange -Input [1]: [d_week_seq#34] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [10]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17] +Right output [1]: [d_week_seq#27] +Arguments: [d_week_seq#5], [d_week_seq#27], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [d_week_seq#5] -Right keys [1]: [d_week_seq#34] -Join type: Inner -Join condition: None +(23) CometProject +Input [11]: [d_week_seq#5, sun_sales#19, mon_sales#20, tue_sales#21, wed_sales#22, thu_sales#23, fri_sales#24, sat_sales#25, s_store_id#18, s_store_name#17, d_week_seq#27] +Arguments: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37], [s_store_name#17 AS s_store_name1#28, d_week_seq#5 AS d_week_seq1#29, s_store_id#18 AS s_store_id1#30, sun_sales#19 AS sun_sales1#31, mon_sales#20 AS mon_sales1#32, tue_sales#21 AS tue_sales1#33, wed_sales#22 AS wed_sales1#34, thu_sales#23 AS thu_sales1#35, fri_sales#24 AS fri_sales1#36, sat_sales#25 AS sat_sales1#37] -(26) Project [codegen id : 6] -Output [10]: [s_store_name#31 AS s_store_name1#35, d_week_seq#5 AS d_week_seq1#36, s_store_id#32 AS s_store_id1#37, sun_sales#22 AS sun_sales1#38, mon_sales#23 AS mon_sales1#39, tue_sales#24 AS tue_sales1#40, wed_sales#25 AS wed_sales1#41, thu_sales#26 AS thu_sales1#42, fri_sales#27 AS fri_sales1#43, sat_sales#28 AS sat_sales1#44] -Input [11]: [d_week_seq#5, sun_sales#22, mon_sales#23, tue_sales#24, wed_sales#25, thu_sales#26, fri_sales#27, sat_sales#28, s_store_id#32, s_store_name#31, d_week_seq#34] +(24) ReusedExchange [Reuses operator id: 10] +Output [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] -(27) ReusedExchange [Reuses operator id: 10] -Output [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] +(25) CometHashAggregate +Input [9]: [d_week_seq#38, ss_store_sk#39, sum#40, sum#41, sum#42, sum#43, sum#44, sum#45, sum#46] +Keys [2]: [d_week_seq#38, ss_store_sk#39] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#47 = Sunday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Monday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Tuesday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Wednesday) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Thursday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Friday ) THEN ss_sales_price#48 END)), sum(UnscaledValue(CASE WHEN (d_day_name#47 = Saturday ) THEN ss_sales_price#48 END))] -(28) CometColumnarToRow [codegen id : 5] -Input [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] - -(29) HashAggregate [codegen id : 5] -Input [9]: [d_week_seq#45, ss_store_sk#46, sum#47, sum#48, sum#49, sum#50, sum#51, sum#52, sum#53] -Keys [2]: [d_week_seq#45, ss_store_sk#46] -Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END)), sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))] -Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END))#15, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END))#16, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END))#17, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END))#18, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END))#19, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END))#20, sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))#21] -Results [9]: [d_week_seq#45, ss_store_sk#46, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Sunday ) THEN ss_sales_price#55 END))#15,17,2) AS sun_sales#56, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Monday ) THEN ss_sales_price#55 END))#16,17,2) AS mon_sales#57, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Tuesday ) THEN ss_sales_price#55 END))#17,17,2) AS tue_sales#58, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Wednesday) THEN ss_sales_price#55 END))#18,17,2) AS wed_sales#59, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Thursday ) THEN ss_sales_price#55 END))#19,17,2) AS thu_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Friday ) THEN ss_sales_price#55 END))#20,17,2) AS fri_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#54 = Saturday ) THEN ss_sales_price#55 END))#21,17,2) AS sat_sales#62] - -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [2]: [s_store_sk#63, s_store_id#64] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +Output [2]: [s_store_sk#49, s_store_id#50] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(31) CometFilter -Input [2]: [s_store_sk#63, s_store_id#64] -Condition : (isnotnull(s_store_sk#63) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#64, 16)))) - -(32) CometProject -Input [2]: [s_store_sk#63, s_store_id#64] -Arguments: [s_store_sk#63, s_store_id#65], [s_store_sk#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#64, 16)) AS s_store_id#65] +(27) CometFilter +Input [2]: [s_store_sk#49, s_store_id#50] +Condition : (isnotnull(s_store_sk#49) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)))) -(33) CometColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#63, s_store_id#65] +(28) CometProject +Input [2]: [s_store_sk#49, s_store_id#50] +Arguments: [s_store_sk#49, s_store_id#51], [s_store_sk#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_store_id#50, 16)) AS s_store_id#51] -(34) BroadcastExchange -Input [2]: [s_store_sk#63, s_store_id#65] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +(29) CometBroadcastExchange +Input [2]: [s_store_sk#49, s_store_id#51] +Arguments: [s_store_sk#49, s_store_id#51] -(35) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#46] -Right keys [1]: [s_store_sk#63] -Join type: Inner -Join condition: None +(30) CometBroadcastHashJoin +Left output [9]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58] +Right output [2]: [s_store_sk#49, s_store_id#51] +Arguments: [ss_store_sk#39], [s_store_sk#49], Inner, BuildRight -(36) Project [codegen id : 5] -Output [9]: [d_week_seq#45, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_id#65] -Input [11]: [d_week_seq#45, ss_store_sk#46, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_sk#63, s_store_id#65] +(31) CometProject +Input [11]: [d_week_seq#38, ss_store_sk#39, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_sk#49, s_store_id#51] +Arguments: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51], [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_month_seq#66, d_week_seq#67] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#59, d_week_seq#60] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] ReadSchema: struct -(38) CometFilter -Input [2]: [d_month_seq#66, d_week_seq#67] -Condition : (((isnotnull(d_month_seq#66) AND (d_month_seq#66 >= 1224)) AND (d_month_seq#66 <= 1235)) AND isnotnull(d_week_seq#67)) +(33) CometFilter +Input [2]: [d_month_seq#59, d_week_seq#60] +Condition : (((isnotnull(d_month_seq#59) AND (d_month_seq#59 >= 1224)) AND (d_month_seq#59 <= 1235)) AND isnotnull(d_week_seq#60)) -(39) CometProject -Input [2]: [d_month_seq#66, d_week_seq#67] -Arguments: [d_week_seq#67], [d_week_seq#67] +(34) CometProject +Input [2]: [d_month_seq#59, d_week_seq#60] +Arguments: [d_week_seq#60], [d_week_seq#60] -(40) CometColumnarToRow [codegen id : 4] -Input [1]: [d_week_seq#67] +(35) CometBroadcastExchange +Input [1]: [d_week_seq#60] +Arguments: [d_week_seq#60] -(41) BroadcastExchange -Input [1]: [d_week_seq#67] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(36) CometBroadcastHashJoin +Left output [9]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51] +Right output [1]: [d_week_seq#60] +Arguments: [d_week_seq#38], [d_week_seq#60], Inner, BuildRight -(42) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [d_week_seq#45] -Right keys [1]: [d_week_seq#67] -Join type: Inner -Join condition: None +(37) CometProject +Input [10]: [d_week_seq#38, sun_sales#52, mon_sales#53, tue_sales#54, wed_sales#55, thu_sales#56, fri_sales#57, sat_sales#58, s_store_id#51, d_week_seq#60] +Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69], [d_week_seq#38 AS d_week_seq2#61, s_store_id#51 AS s_store_id2#62, sun_sales#52 AS sun_sales2#63, mon_sales#53 AS mon_sales2#64, tue_sales#54 AS tue_sales2#65, wed_sales#55 AS wed_sales2#66, thu_sales#56 AS thu_sales2#67, fri_sales#57 AS fri_sales2#68, sat_sales#58 AS sat_sales2#69] -(43) Project [codegen id : 5] -Output [9]: [d_week_seq#45 AS d_week_seq2#68, s_store_id#65 AS s_store_id2#69, sun_sales#56 AS sun_sales2#70, mon_sales#57 AS mon_sales2#71, tue_sales#58 AS tue_sales2#72, wed_sales#59 AS wed_sales2#73, thu_sales#60 AS thu_sales2#74, fri_sales#61 AS fri_sales2#75, sat_sales#62 AS sat_sales2#76] -Input [10]: [d_week_seq#45, sun_sales#56, mon_sales#57, tue_sales#58, wed_sales#59, thu_sales#60, fri_sales#61, sat_sales#62, s_store_id#65, d_week_seq#67] +(38) CometBroadcastExchange +Input [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] -(44) BroadcastExchange -Input [9]: [d_week_seq2#68, s_store_id2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=6] +(39) CometBroadcastHashJoin +Left output [10]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37] +Right output [9]: [d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [s_store_id1#30, d_week_seq1#29], [s_store_id2#62, (d_week_seq2#61 - 52)], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 6] -Left keys [2]: [s_store_id1#37, d_week_seq1#36] -Right keys [2]: [s_store_id2#69, (d_week_seq2#68 - 52)] -Join type: Inner -Join condition: None +(40) CometProject +Input [19]: [s_store_name1#28, d_week_seq1#29, s_store_id1#30, sun_sales1#31, mon_sales1#32, tue_sales1#33, wed_sales1#34, thu_sales1#35, fri_sales1#36, sat_sales1#37, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69] +Arguments: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1#31 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#32 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#33 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#34 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#35 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#36 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#37 / sat_sales2#69) AS (sat_sales1 / sat_sales2)#76] -(46) Project [codegen id : 6] -Output [10]: [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1#38 / sun_sales2#70) AS (sun_sales1 / sun_sales2)#77, (mon_sales1#39 / mon_sales2#71) AS (mon_sales1 / mon_sales2)#78, (tue_sales1#40 / tue_sales2#72) AS (tue_sales1 / tue_sales2)#79, (wed_sales1#41 / wed_sales2#73) AS (wed_sales1 / wed_sales2)#80, (thu_sales1#42 / thu_sales2#74) AS (thu_sales1 / thu_sales2)#81, (fri_sales1#43 / fri_sales2#75) AS (fri_sales1 / fri_sales2)#82, (sat_sales1#44 / sat_sales2#76) AS (sat_sales1 / sat_sales2)#83] -Input [19]: [s_store_name1#35, d_week_seq1#36, s_store_id1#37, sun_sales1#38, mon_sales1#39, tue_sales1#40, wed_sales1#41, thu_sales1#42, fri_sales1#43, sat_sales1#44, d_week_seq2#68, s_store_id2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] +(41) CometTakeOrderedAndProject +Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name1#28 ASC NULLS FIRST,s_store_id1#30 ASC NULLS FIRST,d_week_seq1#29 ASC NULLS FIRST], output=[s_store_name1#28,s_store_id1#30,d_week_seq1#29,(sun_sales1 / sun_sales2)#70,(mon_sales1 / mon_sales2)#71,(tue_sales1 / tue_sales2)#72,(wed_sales1 / wed_sales2)#73,(thu_sales1 / thu_sales2)#74,(fri_sales1 / fri_sales2)#75,(sat_sales1 / sat_sales2)#76]), [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76], 100, 0, [s_store_name1#28 ASC NULLS FIRST, s_store_id1#30 ASC NULLS FIRST, d_week_seq1#29 ASC NULLS FIRST], [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] -(47) TakeOrderedAndProject -Input [10]: [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] -Arguments: 100, [s_store_name1#35 ASC NULLS FIRST, s_store_id1#37 ASC NULLS FIRST, d_week_seq1#36 ASC NULLS FIRST], [s_store_name1#35, s_store_id1#37, d_week_seq1#36, (sun_sales1 / sun_sales2)#77, (mon_sales1 / mon_sales2)#78, (tue_sales1 / tue_sales2)#79, (wed_sales1 / wed_sales2)#80, (thu_sales1 / thu_sales2)#81, (fri_sales1 / fri_sales2)#82, (sat_sales1 / sat_sales2)#83] +(42) CometColumnarToRow [codegen id : 1] +Input [10]: [s_store_name1#28, s_store_id1#30, d_week_seq1#29, (sun_sales1 / sun_sales2)#70, (mon_sales1 / mon_sales2)#71, (tue_sales1 / tue_sales2)#72, (wed_sales1 / wed_sales2)#73, (thu_sales1 / thu_sales2)#74, (fri_sales1 / fri_sales2)#75, (sat_sales1 / sat_sales2)#76] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt index 333907763b..3d50a2131e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/extended.txt @@ -1,58 +1,53 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#1, ss_store_sk#2, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#4 END))#5,17,2) AS sun_sales#6, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#4 END))#7,17,2) AS mon_sales#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#4 END))#9,17,2) AS tue_sales#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#4 END))#11,17,2) AS wed_sales#12, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#4 END))#13,17,2) AS thu_sales#14, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#4 END))#15,17,2) AS fri_sales#16, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#4 END))#17,17,2) AS sat_sales#18)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- 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 - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_week_seq#19, ss_store_sk#20, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Sunday ) THEN ss_sales_price#22 END))#5,17,2) AS sun_sales#23, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Monday ) THEN ss_sales_price#22 END))#7,17,2) AS mon_sales#24, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Tuesday ) THEN ss_sales_price#22 END))#9,17,2) AS tue_sales#25, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Wednesday) THEN ss_sales_price#22 END))#11,17,2) AS wed_sales#26, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Thursday ) THEN ss_sales_price#22 END))#13,17,2) AS thu_sales#27, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Friday ) THEN ss_sales_price#22 END))#15,17,2) AS fri_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#21 = Saturday ) THEN ss_sales_price#22 END))#17,17,2) AS sat_sales#29)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 50 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 50 out of 50 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt index 867d0d888e..bca9c6463d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] - WholeStageCodegen (6) - Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] - Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + CometBroadcastHashJoin [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometProject [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [s_store_name1,d_week_seq1,s_store_id1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id,s_store_name] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] CometExchange [d_week_seq,ss_store_sk] #1 CometHashAggregate [d_day_name,ss_sales_price] [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] CometProject [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] @@ -19,46 +19,26 @@ TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_s CometProject [d_day_name] [d_date_sk,d_week_seq,d_day_name] CometFilter [d_date_sk,d_week_seq,d_day_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] - CometFilter [s_store_sk,s_store_id,s_store_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (5) - Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] - BroadcastHashJoin [ss_store_sk,s_store_sk] - HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - 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 #7 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [d_week_seq] - CometFilter [d_month_seq,d_week_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [s_store_sk,s_store_id,s_store_name] #3 + CometProject [s_store_id] [s_store_sk,s_store_id,s_store_name] + CometFilter [s_store_sk,s_store_id,s_store_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + CometBroadcastExchange [d_week_seq] #4 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + CometBroadcastExchange [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] #5 + CometProject [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq2,s_store_id2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] + CometBroadcastHashJoin [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,d_week_seq] + CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + CometBroadcastHashJoin [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_sk,s_store_id] + CometHashAggregate [sum,sum,sum,sum,sum,sum,sum] [d_week_seq,ss_store_sk,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END))] + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + 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] + CometBroadcastExchange [d_week_seq] #7 + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt index e3b4e28884..1777ebee23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/explain.txt @@ -1,69 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometColumnarExchange (62) - +- * HashAggregate (61) - +- Union (60) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (37) - : : +- CometBroadcastHashJoin (36) - : : :- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (59) - +- * CometColumnarToRow (58) - +- CometExchange (57) - +- CometHashAggregate (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (49) - : : +- CometBroadcastHashJoin (48) - : : :- CometFilter (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* CometColumnarToRow (62) ++- CometTakeOrderedAndProject (61) + +- CometHashAggregate (60) + +- CometExchange (59) + +- CometHashAggregate (58) + +- CometUnion (57) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometProject (53) + +- CometBroadcastHashJoin (52) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometFilter (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) + : : +- ReusedExchange (45) + : +- ReusedExchange (48) + +- ReusedExchange (51) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -195,211 +192,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_item_id#16, sum#17] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#16, sum#17] - -(29) HashAggregate [codegen id : 1] +(28) CometHashAggregate Input [2]: [i_item_id#16, sum#17] Keys [1]: [i_item_id#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [2]: [i_item_id#16 AS i_item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS total_sales#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : (isnotnull(cs_bill_addr_sk#21) AND isnotnull(cs_item_sk#22)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#26] +(30) CometFilter +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) -(33) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#24], [d_date_sk#26], Inner, BuildRight +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(34) CometProject -Input [5]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#26] -Arguments: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23], [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#27] +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(36) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Right output [1]: [ca_address_sk#27] -Arguments: [cs_bill_addr_sk#21], [ca_address_sk#27], Inner, BuildRight +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(37) CometProject -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, ca_address_sk#27] -Arguments: [cs_item_sk#22, cs_ext_sales_price#23], [cs_item_sk#22, cs_ext_sales_price#23] +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#28, i_item_id#29] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(39) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#22], [i_item_sk#28], Inner, BuildRight +(37) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#25, i_item_id#26] -(40) CometProject -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#23, i_item_id#29], [cs_ext_sales_price#23, i_item_id#29] +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight -(41) CometHashAggregate -Input [2]: [cs_ext_sales_price#23, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(42) CometExchange -Input [2]: [i_item_id#29, sum#30] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(43) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] +(41) CometExchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(44) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#31] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#31,17,2) AS total_sales#32] +(42) CometHashAggregate +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(44) CometFilter +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) -(47) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#38] +(45) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(48) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(49) CometProject -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38] -Arguments: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35], [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +(47) CometProject +Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] +Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -(50) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#39] +(48) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#34] -(51) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Right output [1]: [ca_address_sk#39] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#39], Inner, BuildRight +(49) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] +Right output [1]: [ca_address_sk#34] +Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight -(52) CometProject -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#39] -Arguments: [ws_item_sk#33, ws_ext_sales_price#35], [ws_item_sk#33, ws_ext_sales_price#35] +(50) CometProject +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] +Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#40, i_item_id#41] +(51) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#35, i_item_id#36] -(54) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Right output [2]: [i_item_sk#40, i_item_id#41] -Arguments: [ws_item_sk#33], [i_item_sk#40], Inner, BuildRight +(52) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] +Right output [2]: [i_item_sk#35, i_item_id#36] +Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight -(55) CometProject -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#40, i_item_id#41] -Arguments: [ws_ext_sales_price#35, i_item_id#41], [ws_ext_sales_price#35, i_item_id#41] +(53) CometProject +Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] +Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] + +(54) CometHashAggregate +Input [2]: [ws_ext_sales_price#30, i_item_id#36] +Keys [1]: [i_item_id#36] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] + +(55) CometExchange +Input [2]: [i_item_id#36, sum#37] +Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (56) CometHashAggregate -Input [2]: [ws_ext_sales_price#35, i_item_id#41] -Keys [1]: [i_item_id#41] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(57) CometExchange -Input [2]: [i_item_id#41, sum#42] -Arguments: hashpartitioning(i_item_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(58) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] - -(59) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] -Keys [1]: [i_item_id#41] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#41, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS total_sales#44] - -(60) Union - -(61) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#19, total_sales#20] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(total_sales#20)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [3]: [i_item_id#19, sum#47, isEmpty#48] - -(62) CometColumnarExchange -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(63) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] - -(64) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(total_sales#20)] -Aggregate Attributes [1]: [sum(total_sales#20)#49] -Results [2]: [i_item_id#19, sum(total_sales#20)#49 AS total_sales#50] - -(65) TakeOrderedAndProject -Input [2]: [i_item_id#19, total_sales#50] -Arguments: 100, [i_item_id#19 ASC NULLS FIRST, total_sales#50 ASC NULLS FIRST], [i_item_id#19, total_sales#50] +Input [2]: [i_item_id#36, sum#37] +Keys [1]: [i_item_id#36] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] + +(57) CometUnion +Child 0 Input [2]: [i_item_id#38, total_sales#39] +Child 1 Input [2]: [i_item_id#26, total_sales#40] +Child 2 Input [2]: [i_item_id#36, total_sales#41] + +(58) CometHashAggregate +Input [2]: [i_item_id#38, total_sales#39] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(total_sales#39)] + +(59) CometExchange +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(60) CometHashAggregate +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(total_sales#39)] + +(61) CometTakeOrderedAndProject +Input [2]: [i_item_id#38, total_sales#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] + +(62) CometColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#38, total_sales#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (67) ++- * CometColumnarToRow (66) + +- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(64) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) -(68) CometProject +(65) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(69) CometColumnarToRow [codegen id : 1] +(66) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(70) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt index 1b9273b540..36b9a8306d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/extended.txt @@ -1,103 +1,100 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS i_item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS total_sales#5)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#7))#8,17,2) AS total_sales#9)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#11))#12,17,2) AS total_sales#13)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 85 out of 96 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt index f3ab606ff1..086da3b776 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60.native_iceberg_compat/simplified.txt @@ -1,85 +1,74 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (5) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,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] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] + CometExchange [i_item_id] #1 + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] + CometUnion [i_item_id,total_sales] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,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] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt index e3b4e28884..1777ebee23 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt @@ -1,69 +1,66 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometColumnarExchange (62) - +- * HashAggregate (61) - +- Union (60) - :- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- 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.customer_address (9) - : +- CometBroadcastExchange (23) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) - : +- CometBroadcastExchange (20) - : +- CometProject (19) - : +- CometFilter (18) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) - :- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometProject (40) - : +- CometBroadcastHashJoin (39) - : :- CometProject (37) - : : +- CometBroadcastHashJoin (36) - : : :- CometProject (34) - : : : +- CometBroadcastHashJoin (33) - : : : :- CometFilter (31) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (30) - : : : +- ReusedExchange (32) - : : +- ReusedExchange (35) - : +- ReusedExchange (38) - +- * HashAggregate (59) - +- * CometColumnarToRow (58) - +- CometExchange (57) - +- CometHashAggregate (56) - +- CometProject (55) - +- CometBroadcastHashJoin (54) - :- CometProject (52) - : +- CometBroadcastHashJoin (51) - : :- CometProject (49) - : : +- CometBroadcastHashJoin (48) - : : :- CometFilter (46) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (45) - : : +- ReusedExchange (47) - : +- ReusedExchange (50) - +- ReusedExchange (53) +* CometColumnarToRow (62) ++- CometTakeOrderedAndProject (61) + +- CometHashAggregate (60) + +- CometExchange (59) + +- CometHashAggregate (58) + +- CometUnion (57) + :- CometHashAggregate (28) + : +- CometExchange (27) + : +- CometHashAggregate (26) + : +- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- 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.customer_address (9) + : +- CometBroadcastExchange (23) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (15) + : +- CometBroadcastExchange (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (17) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometProject (33) + : : : +- CometBroadcastHashJoin (32) + : : : :- CometFilter (30) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (31) + : : +- ReusedExchange (34) + : +- ReusedExchange (37) + +- CometHashAggregate (56) + +- CometExchange (55) + +- CometHashAggregate (54) + +- CometProject (53) + +- CometBroadcastHashJoin (52) + :- CometProject (50) + : +- CometBroadcastHashJoin (49) + : :- CometProject (47) + : : +- CometBroadcastHashJoin (46) + : : :- CometFilter (44) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (43) + : : +- ReusedExchange (45) + : +- ReusedExchange (48) + +- ReusedExchange (51) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -195,211 +192,195 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Input [2]: [i_item_id#16, sum#17] Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 1] -Input [2]: [i_item_id#16, sum#17] - -(29) HashAggregate [codegen id : 1] +(28) CometHashAggregate Input [2]: [i_item_id#16, sum#17] Keys [1]: [i_item_id#16] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [2]: [i_item_id#16 AS i_item_id#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS total_sales#20] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#24), dynamicpruningexpression(cs_sold_date_sk#24 IN dynamicpruning#25)] +PartitionFilters: [isnotnull(cs_sold_date_sk#21), dynamicpruningexpression(cs_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Condition : (isnotnull(cs_bill_addr_sk#21) AND isnotnull(cs_item_sk#22)) - -(32) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#26] +(30) CometFilter +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Condition : (isnotnull(cs_bill_addr_sk#18) AND isnotnull(cs_item_sk#19)) -(33) CometBroadcastHashJoin -Left output [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24] -Right output [1]: [d_date_sk#26] -Arguments: [cs_sold_date_sk#24], [d_date_sk#26], Inner, BuildRight +(31) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#23] -(34) CometProject -Input [5]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, cs_sold_date_sk#24, d_date_sk#26] -Arguments: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23], [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] +(32) CometBroadcastHashJoin +Left output [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [cs_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight -(35) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#27] +(33) CometProject +Input [5]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, cs_sold_date_sk#21, d_date_sk#23] +Arguments: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20], [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] -(36) CometBroadcastHashJoin -Left output [3]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23] -Right output [1]: [ca_address_sk#27] -Arguments: [cs_bill_addr_sk#21], [ca_address_sk#27], Inner, BuildRight +(34) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#24] -(37) CometProject -Input [4]: [cs_bill_addr_sk#21, cs_item_sk#22, cs_ext_sales_price#23, ca_address_sk#27] -Arguments: [cs_item_sk#22, cs_ext_sales_price#23], [cs_item_sk#22, cs_ext_sales_price#23] +(35) CometBroadcastHashJoin +Left output [3]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20] +Right output [1]: [ca_address_sk#24] +Arguments: [cs_bill_addr_sk#18], [ca_address_sk#24], Inner, BuildRight -(38) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#28, i_item_id#29] +(36) CometProject +Input [4]: [cs_bill_addr_sk#18, cs_item_sk#19, cs_ext_sales_price#20, ca_address_sk#24] +Arguments: [cs_item_sk#19, cs_ext_sales_price#20], [cs_item_sk#19, cs_ext_sales_price#20] -(39) CometBroadcastHashJoin -Left output [2]: [cs_item_sk#22, cs_ext_sales_price#23] -Right output [2]: [i_item_sk#28, i_item_id#29] -Arguments: [cs_item_sk#22], [i_item_sk#28], Inner, BuildRight +(37) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#25, i_item_id#26] -(40) CometProject -Input [4]: [cs_item_sk#22, cs_ext_sales_price#23, i_item_sk#28, i_item_id#29] -Arguments: [cs_ext_sales_price#23, i_item_id#29], [cs_ext_sales_price#23, i_item_id#29] +(38) CometBroadcastHashJoin +Left output [2]: [cs_item_sk#19, cs_ext_sales_price#20] +Right output [2]: [i_item_sk#25, i_item_id#26] +Arguments: [cs_item_sk#19], [i_item_sk#25], Inner, BuildRight -(41) CometHashAggregate -Input [2]: [cs_ext_sales_price#23, i_item_id#29] -Keys [1]: [i_item_id#29] -Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#23))] +(39) CometProject +Input [4]: [cs_item_sk#19, cs_ext_sales_price#20, i_item_sk#25, i_item_id#26] +Arguments: [cs_ext_sales_price#20, i_item_id#26], [cs_ext_sales_price#20, i_item_id#26] -(42) CometExchange -Input [2]: [i_item_id#29, sum#30] -Arguments: hashpartitioning(i_item_id#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometHashAggregate +Input [2]: [cs_ext_sales_price#20, i_item_id#26] +Keys [1]: [i_item_id#26] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#20))] -(43) CometColumnarToRow [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] +(41) CometExchange +Input [2]: [i_item_id#26, sum#27] +Arguments: hashpartitioning(i_item_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(44) HashAggregate [codegen id : 2] -Input [2]: [i_item_id#29, sum#30] -Keys [1]: [i_item_id#29] -Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#23))#31] -Results [2]: [i_item_id#29, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#23))#31,17,2) AS total_sales#32] +(42) CometHashAggregate +Input [2]: [i_item_id#26, sum#27] +Keys [1]: [i_item_id#26] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#20))] -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#36), dynamicpruningexpression(ws_sold_date_sk#36 IN dynamicpruning#37)] +PartitionFilters: [isnotnull(ws_sold_date_sk#31), dynamicpruningexpression(ws_sold_date_sk#31 IN dynamicpruning#32)] PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] ReadSchema: struct -(46) CometFilter -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Condition : (isnotnull(ws_bill_addr_sk#34) AND isnotnull(ws_item_sk#33)) +(44) CometFilter +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Condition : (isnotnull(ws_bill_addr_sk#29) AND isnotnull(ws_item_sk#28)) -(47) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#38] +(45) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(48) CometBroadcastHashJoin -Left output [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36] -Right output [1]: [d_date_sk#38] -Arguments: [ws_sold_date_sk#36], [d_date_sk#38], Inner, BuildRight +(46) CometBroadcastHashJoin +Left output [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31] +Right output [1]: [d_date_sk#33] +Arguments: [ws_sold_date_sk#31], [d_date_sk#33], Inner, BuildRight -(49) CometProject -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ws_sold_date_sk#36, d_date_sk#38] -Arguments: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35], [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +(47) CometProject +Input [5]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ws_sold_date_sk#31, d_date_sk#33] +Arguments: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30], [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] -(50) ReusedExchange [Reuses operator id: 12] -Output [1]: [ca_address_sk#39] +(48) ReusedExchange [Reuses operator id: 12] +Output [1]: [ca_address_sk#34] -(51) CometBroadcastHashJoin -Left output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Right output [1]: [ca_address_sk#39] -Arguments: [ws_bill_addr_sk#34], [ca_address_sk#39], Inner, BuildRight +(49) CometBroadcastHashJoin +Left output [3]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30] +Right output [1]: [ca_address_sk#34] +Arguments: [ws_bill_addr_sk#29], [ca_address_sk#34], Inner, BuildRight -(52) CometProject -Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#39] -Arguments: [ws_item_sk#33, ws_ext_sales_price#35], [ws_item_sk#33, ws_ext_sales_price#35] +(50) CometProject +Input [4]: [ws_item_sk#28, ws_bill_addr_sk#29, ws_ext_sales_price#30, ca_address_sk#34] +Arguments: [ws_item_sk#28, ws_ext_sales_price#30], [ws_item_sk#28, ws_ext_sales_price#30] -(53) ReusedExchange [Reuses operator id: 23] -Output [2]: [i_item_sk#40, i_item_id#41] +(51) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#35, i_item_id#36] -(54) CometBroadcastHashJoin -Left output [2]: [ws_item_sk#33, ws_ext_sales_price#35] -Right output [2]: [i_item_sk#40, i_item_id#41] -Arguments: [ws_item_sk#33], [i_item_sk#40], Inner, BuildRight +(52) CometBroadcastHashJoin +Left output [2]: [ws_item_sk#28, ws_ext_sales_price#30] +Right output [2]: [i_item_sk#35, i_item_id#36] +Arguments: [ws_item_sk#28], [i_item_sk#35], Inner, BuildRight -(55) CometProject -Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#40, i_item_id#41] -Arguments: [ws_ext_sales_price#35, i_item_id#41], [ws_ext_sales_price#35, i_item_id#41] +(53) CometProject +Input [4]: [ws_item_sk#28, ws_ext_sales_price#30, i_item_sk#35, i_item_id#36] +Arguments: [ws_ext_sales_price#30, i_item_id#36], [ws_ext_sales_price#30, i_item_id#36] + +(54) CometHashAggregate +Input [2]: [ws_ext_sales_price#30, i_item_id#36] +Keys [1]: [i_item_id#36] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#30))] + +(55) CometExchange +Input [2]: [i_item_id#36, sum#37] +Arguments: hashpartitioning(i_item_id#36, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (56) CometHashAggregate -Input [2]: [ws_ext_sales_price#35, i_item_id#41] -Keys [1]: [i_item_id#41] -Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] - -(57) CometExchange -Input [2]: [i_item_id#41, sum#42] -Arguments: hashpartitioning(i_item_id#41, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(58) CometColumnarToRow [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] - -(59) HashAggregate [codegen id : 3] -Input [2]: [i_item_id#41, sum#42] -Keys [1]: [i_item_id#41] -Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#43] -Results [2]: [i_item_id#41, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#43,17,2) AS total_sales#44] - -(60) Union - -(61) HashAggregate [codegen id : 4] -Input [2]: [i_item_id#19, total_sales#20] -Keys [1]: [i_item_id#19] -Functions [1]: [partial_sum(total_sales#20)] -Aggregate Attributes [2]: [sum#45, isEmpty#46] -Results [3]: [i_item_id#19, sum#47, isEmpty#48] - -(62) CometColumnarExchange -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(63) CometColumnarToRow [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] - -(64) HashAggregate [codegen id : 5] -Input [3]: [i_item_id#19, sum#47, isEmpty#48] -Keys [1]: [i_item_id#19] -Functions [1]: [sum(total_sales#20)] -Aggregate Attributes [1]: [sum(total_sales#20)#49] -Results [2]: [i_item_id#19, sum(total_sales#20)#49 AS total_sales#50] - -(65) TakeOrderedAndProject -Input [2]: [i_item_id#19, total_sales#50] -Arguments: 100, [i_item_id#19 ASC NULLS FIRST, total_sales#50 ASC NULLS FIRST], [i_item_id#19, total_sales#50] +Input [2]: [i_item_id#36, sum#37] +Keys [1]: [i_item_id#36] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#30))] + +(57) CometUnion +Child 0 Input [2]: [i_item_id#38, total_sales#39] +Child 1 Input [2]: [i_item_id#26, total_sales#40] +Child 2 Input [2]: [i_item_id#36, total_sales#41] + +(58) CometHashAggregate +Input [2]: [i_item_id#38, total_sales#39] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(total_sales#39)] + +(59) CometExchange +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(60) CometHashAggregate +Input [3]: [i_item_id#38, sum#42, isEmpty#43] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(total_sales#39)] + +(61) CometTakeOrderedAndProject +Input [2]: [i_item_id#38, total_sales#44] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_item_id#38 ASC NULLS FIRST,total_sales#44 ASC NULLS FIRST], output=[i_item_id#38,total_sales#44]), [i_item_id#38, total_sales#44], 100, 0, [i_item_id#38 ASC NULLS FIRST, total_sales#44 ASC NULLS FIRST], [i_item_id#38, total_sales#44] + +(62) CometColumnarToRow [codegen id : 1] +Input [2]: [i_item_id#38, total_sales#44] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (67) ++- * CometColumnarToRow (66) + +- CometProject (65) + +- CometFilter (64) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (63) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(63) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#6, d_year#7, d_moy#8] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(64) CometFilter Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 9)) AND isnotnull(d_date_sk#6)) -(68) CometProject +(65) CometProject Input [3]: [d_date_sk#6, d_year#7, d_moy#8] Arguments: [d_date_sk#6], [d_date_sk#6] -(69) CometColumnarToRow [codegen id : 1] +(66) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(70) BroadcastExchange +(67) BroadcastExchange Input [1]: [d_date_sk#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#24 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#21 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#36 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#31 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt index 1b9273b540..36b9a8306d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/extended.txt @@ -1,103 +1,100 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1 AS i_item_id#2, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2) AS total_sales#5)] - : +- CometColumnarToRow - : +- 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.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#6, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#7))#8,17,2) AS total_sales#9)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#11))#12,17,2) AS total_sales#13)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 85 out of 96 eligible operators (88%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 92 out of 96 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt index f3ab606ff1..086da3b776 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -1,85 +1,74 @@ -TakeOrderedAndProject [i_item_id,total_sales] - WholeStageCodegen (5) - HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_item_id] #1 - WholeStageCodegen (4) - HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),i_item_id,total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #2 - CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] - CometProject [ss_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] - CometProject [ss_item_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] - CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [i_item_sk,i_item_id] #6 - CometProject [i_item_id] [i_item_sk,i_item_id] - CometBroadcastHashJoin [i_item_sk,i_item_id,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] - CometBroadcastExchange [i_item_id] #7 - CometProject [i_item_id] [i_item_id] - CometFilter [i_item_id,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] - WholeStageCodegen (2) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #8 - CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] - CometProject [cs_ext_sales_price,i_item_id] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] - CometProject [cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] - CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] - CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 - WholeStageCodegen (3) - HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id] #9 - CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] - CometProject [ws_ext_sales_price,i_item_id] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] - CometProject [ws_item_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] - CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] - CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - ReusedExchange [ca_address_sk] #5 - ReusedExchange [i_item_sk,i_item_id] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,total_sales] + CometHashAggregate [sum,isEmpty] [i_item_id,total_sales,sum(total_sales)] + CometExchange [i_item_id] #1 + CometHashAggregate [total_sales] [i_item_id,sum,isEmpty] + CometUnion [i_item_id,total_sales] + CometHashAggregate [sum] [i_item_id,total_sales,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id] #2 + CometHashAggregate [ss_ext_sales_price] [i_item_id,sum] + CometProject [ss_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,i_item_sk,i_item_id] + CometProject [ss_item_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ca_address_sk] + CometProject [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,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 [ca_address_sk] #5 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [i_item_sk,i_item_id] #6 + CometProject [i_item_id] [i_item_sk,i_item_id] + CometBroadcastHashJoin [i_item_sk,i_item_id,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] + CometBroadcastExchange [i_item_id] #7 + CometProject [i_item_id] [i_item_id] + CometFilter [i_item_id,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_id,i_category] + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id] #8 + CometHashAggregate [cs_ext_sales_price] [i_item_id,sum] + CometProject [cs_ext_sales_price,i_item_id] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,i_item_sk,i_item_id] + CometProject [cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,ca_address_sk] + CometProject [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + CometBroadcastHashJoin [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 + CometHashAggregate [sum] [i_item_id,total_sales,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id] #9 + CometHashAggregate [ws_ext_sales_price] [i_item_id,sum] + CometProject [ws_ext_sales_price,i_item_id] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,i_item_sk,i_item_id] + CometProject [ws_item_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ca_address_sk] + CometProject [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + CometBroadcastHashJoin [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + ReusedExchange [ca_address_sk] #5 + ReusedExchange [i_item_sk,i_item_id] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt index e793ac1245..9465cc9c1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) + :- * CometColumnarToRow (41) + : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) @@ -43,8 +43,8 @@ : +- CometFilter (33) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * HashAggregate (62) - +- * CometColumnarToRow (61) + +- * CometColumnarToRow (62) + +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) +- CometProject (58) @@ -251,109 +251,105 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [1]: [sum#23] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarToRow [codegen id : 2] -Input [1]: [sum#23] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#24,17,2) AS promotions#25] + +(41) CometColumnarToRow [codegen id : 2] +Input [1]: [promotions#24] (42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (43) CometFilter -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : ((isnotnull(ss_store_sk#28) AND isnotnull(ss_customer_sk#27)) AND isnotnull(ss_item_sk#26)) +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#32] +Output [1]: [s_store_sk#31] (45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [s_store_sk#32] -Arguments: [ss_store_sk#28], [s_store_sk#32], Inner, BuildRight +Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [s_store_sk#31] +Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight (46) CometProject -Input [6]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30, s_store_sk#32] -Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] (47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#33] +Output [1]: [d_date_sk#32] (48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#30], [d_date_sk#33], Inner, BuildRight +Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [d_date_sk#32] +Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight (49) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#33] -Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] (50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#34, c_current_addr_sk#35] +Output [2]: [c_customer_sk#33, c_current_addr_sk#34] (51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] -Right output [2]: [c_customer_sk#34, c_current_addr_sk#35] -Arguments: [ss_customer_sk#27], [c_customer_sk#34], Inner, BuildRight +Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight (52) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, c_customer_sk#34, c_current_addr_sk#35] -Arguments: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35], [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] (53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#36] +Output [1]: [ca_address_sk#35] (54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] -Right output [1]: [ca_address_sk#36] -Arguments: [c_current_addr_sk#35], [ca_address_sk#36], Inner, BuildRight +Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Right output [1]: [ca_address_sk#35] +Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight (55) CometProject -Input [4]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35, ca_address_sk#36] -Arguments: [ss_item_sk#26, ss_ext_sales_price#29], [ss_item_sk#26, ss_ext_sales_price#29] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] (56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#36] (57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#26, ss_ext_sales_price#29] -Right output [1]: [i_item_sk#37] -Arguments: [ss_item_sk#26], [i_item_sk#37], Inner, BuildRight +Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Right output [1]: [i_item_sk#36] +Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight (58) CometProject -Input [3]: [ss_item_sk#26, ss_ext_sales_price#29, i_item_sk#37] -Arguments: [ss_ext_sales_price#29], [ss_ext_sales_price#29] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] (59) CometHashAggregate -Input [1]: [ss_ext_sales_price#29] +Input [1]: [ss_ext_sales_price#28] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] (60) CometExchange -Input [1]: [sum#38] +Input [1]: [sum#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#38] - -(62) HashAggregate [codegen id : 1] -Input [1]: [sum#38] +(61) CometHashAggregate +Input [1]: [sum#37] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#29))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#39,17,2) AS total#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] + +(62) CometColumnarToRow [codegen id : 1] +Input [1]: [total#38] (63) BroadcastExchange -Input [1]: [total#40] +Input [1]: [total#38] Arguments: IdentityBroadcastMode, [plan_id=3] (64) BroadcastNestedLoopJoin [codegen id : 2] @@ -361,8 +357,8 @@ Join type: Inner Join condition: None (65) Project [codegen id : 2] -Output [3]: [promotions#25, total#40, ((cast(promotions#25 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] -Input [2]: [promotions#25, total#40] +Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] +Input [2]: [promotions#24, total#38] ===== Subqueries ===== @@ -396,6 +392,6 @@ Input [1]: [d_date_sk#14] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt index 22b4a89de7..fb4bf01d07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ Project +- BroadcastNestedLoopJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) AS promotions#3)] - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -48,8 +48,8 @@ Project : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS total#6)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -85,4 +85,4 @@ Project +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 75 out of 83 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt index ad3a7005cc..6ebe25b753 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61.native_iceberg_compat/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] @@ -54,9 +54,9 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt index e793ac1245..9465cc9c1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == * Project (65) +- * BroadcastNestedLoopJoin Inner BuildRight (64) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) + :- * CometColumnarToRow (41) + : +- CometHashAggregate (40) : +- CometExchange (39) : +- CometHashAggregate (38) : +- CometProject (37) @@ -43,8 +43,8 @@ : +- CometFilter (33) : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (32) +- BroadcastExchange (63) - +- * HashAggregate (62) - +- * CometColumnarToRow (61) + +- * CometColumnarToRow (62) + +- CometHashAggregate (61) +- CometExchange (60) +- CometHashAggregate (59) +- CometProject (58) @@ -251,109 +251,105 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] Input [1]: [sum#23] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarToRow [codegen id : 2] -Input [1]: [sum#23] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [1]: [sum#23] Keys: [] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#24,17,2) AS promotions#25] + +(41) CometColumnarToRow [codegen id : 2] +Input [1]: [promotions#24] (42) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#30), dynamicpruningexpression(ss_sold_date_sk#30 IN dynamicpruning#31)] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] ReadSchema: struct (43) CometFilter -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Condition : ((isnotnull(ss_store_sk#28) AND isnotnull(ss_customer_sk#27)) AND isnotnull(ss_item_sk#26)) +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) (44) ReusedExchange [Reuses operator id: 6] -Output [1]: [s_store_sk#32] +Output [1]: [s_store_sk#31] (45) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [s_store_sk#32] -Arguments: [ss_store_sk#28], [s_store_sk#32], Inner, BuildRight +Left output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [s_store_sk#31] +Arguments: [ss_store_sk#27], [s_store_sk#31], Inner, BuildRight (46) CometProject -Input [6]: [ss_item_sk#26, ss_customer_sk#27, ss_store_sk#28, ss_ext_sales_price#29, ss_sold_date_sk#30, s_store_sk#32] -Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] +Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] (47) ReusedExchange [Reuses operator id: 18] -Output [1]: [d_date_sk#33] +Output [1]: [d_date_sk#32] (48) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30] -Right output [1]: [d_date_sk#33] -Arguments: [ss_sold_date_sk#30], [d_date_sk#33], Inner, BuildRight +Left output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Right output [1]: [d_date_sk#32] +Arguments: [ss_sold_date_sk#29], [d_date_sk#32], Inner, BuildRight (49) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, ss_sold_date_sk#30, d_date_sk#33] -Arguments: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29], [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] +Arguments: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28], [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] (50) ReusedExchange [Reuses operator id: 23] -Output [2]: [c_customer_sk#34, c_current_addr_sk#35] +Output [2]: [c_customer_sk#33, c_current_addr_sk#34] (51) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29] -Right output [2]: [c_customer_sk#34, c_current_addr_sk#35] -Arguments: [ss_customer_sk#27], [c_customer_sk#34], Inner, BuildRight +Left output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Right output [2]: [c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_customer_sk#26], [c_customer_sk#33], Inner, BuildRight (52) CometProject -Input [5]: [ss_item_sk#26, ss_customer_sk#27, ss_ext_sales_price#29, c_customer_sk#34, c_current_addr_sk#35] -Arguments: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35], [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34], [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] (53) ReusedExchange [Reuses operator id: 29] -Output [1]: [ca_address_sk#36] +Output [1]: [ca_address_sk#35] (54) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35] -Right output [1]: [ca_address_sk#36] -Arguments: [c_current_addr_sk#35], [ca_address_sk#36], Inner, BuildRight +Left output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Right output [1]: [ca_address_sk#35] +Arguments: [c_current_addr_sk#34], [ca_address_sk#35], Inner, BuildRight (55) CometProject -Input [4]: [ss_item_sk#26, ss_ext_sales_price#29, c_current_addr_sk#35, ca_address_sk#36] -Arguments: [ss_item_sk#26, ss_ext_sales_price#29], [ss_item_sk#26, ss_ext_sales_price#29] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] +Arguments: [ss_item_sk#25, ss_ext_sales_price#28], [ss_item_sk#25, ss_ext_sales_price#28] (56) ReusedExchange [Reuses operator id: 35] -Output [1]: [i_item_sk#37] +Output [1]: [i_item_sk#36] (57) CometBroadcastHashJoin -Left output [2]: [ss_item_sk#26, ss_ext_sales_price#29] -Right output [1]: [i_item_sk#37] -Arguments: [ss_item_sk#26], [i_item_sk#37], Inner, BuildRight +Left output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Right output [1]: [i_item_sk#36] +Arguments: [ss_item_sk#25], [i_item_sk#36], Inner, BuildRight (58) CometProject -Input [3]: [ss_item_sk#26, ss_ext_sales_price#29, i_item_sk#37] -Arguments: [ss_ext_sales_price#29], [ss_ext_sales_price#29] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] +Arguments: [ss_ext_sales_price#28], [ss_ext_sales_price#28] (59) CometHashAggregate -Input [1]: [ss_ext_sales_price#29] +Input [1]: [ss_ext_sales_price#28] Keys: [] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#29))] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] (60) CometExchange -Input [1]: [sum#38] +Input [1]: [sum#37] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(61) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#38] - -(62) HashAggregate [codegen id : 1] -Input [1]: [sum#38] +(61) CometHashAggregate +Input [1]: [sum#37] Keys: [] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#29))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#29))#39] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#29))#39,17,2) AS total#40] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] + +(62) CometColumnarToRow [codegen id : 1] +Input [1]: [total#38] (63) BroadcastExchange -Input [1]: [total#40] +Input [1]: [total#38] Arguments: IdentityBroadcastMode, [plan_id=3] (64) BroadcastNestedLoopJoin [codegen id : 2] @@ -361,8 +357,8 @@ Join type: Inner Join condition: None (65) Project [codegen id : 2] -Output [3]: [promotions#25, total#40, ((cast(promotions#25 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] -Input [2]: [promotions#25, total#40] +Output [3]: [promotions#24, total#38, ((cast(promotions#24 as decimal(15,4)) / cast(total#38 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#39] +Input [2]: [promotions#24, total#38] ===== Subqueries ===== @@ -396,6 +392,6 @@ Input [1]: [d_date_sk#14] Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#30 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 42 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt index 22b4a89de7..fb4bf01d07 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/extended.txt @@ -1,7 +1,7 @@ Project +- BroadcastNestedLoopJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) AS promotions#3)] - : +- CometColumnarToRow + :- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -48,8 +48,8 @@ Project : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS total#6)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -85,4 +85,4 @@ Project +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 75 out of 83 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 77 out of 83 eligible operators (92%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt index ad3a7005cc..6ebe25b753 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -1,9 +1,9 @@ WholeStageCodegen (2) Project [promotions,total] BroadcastNestedLoopJoin - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [promotions,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #1 CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] @@ -54,9 +54,9 @@ WholeStageCodegen (2) InputAdapter BroadcastExchange #9 WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [total,sum(UnscaledValue(ss_ext_sales_price))] CometExchange #10 CometHashAggregate [ss_ext_sales_price] [sum] CometProject [ss_ext_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt index d927679b54..608bf20b3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Input [3]: [i_manager_id#5, d_moy#17, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, d_moy#17, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [3]: [i_manager_id#5, d_moy#17, sum#19] Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] +(24) CometSort +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] +(26) Window +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(28) Filter [codegen id : 3] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 2] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +(28) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#15, d_moy#17] 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-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt index fb954d652e..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt index 6688a7a467..02166879a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + 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 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index d927679b54..608bf20b3e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#12))] Input [3]: [i_manager_id#5, d_moy#17, sum#19] Arguments: hashpartitioning(i_manager_id#5, d_moy#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [3]: [i_manager_id#5, d_moy#17, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [3]: [i_manager_id#5, d_moy#17, sum#19] Keys [2]: [i_manager_id#5, d_moy#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] -Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] +(24) CometSort +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] +(26) Window +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(28) Filter [codegen id : 3] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END +(27) Filter [codegen id : 2] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +(28) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -(30) TakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(29) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Condition : (d_month_seq#16 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#15, d_month_seq#16, d_moy#17] Arguments: [d_date_sk#15, d_moy#17], [d_date_sk#15, d_moy#17] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#15, d_moy#17] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#15, d_moy#17] 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-v1_4-spark4_0/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt index fb954d652e..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index 6688a7a467..02166879a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_manager_id,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_manager_id] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - 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 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + 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 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt index 44c02a061d..806525a4e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/explain.txt @@ -1,189 +1,180 @@ == Physical Plan == -* CometColumnarToRow (185) -+- CometSort (184) - +- CometExchange (183) - +- CometProject (182) - +- CometSortMergeJoin (181) - :- CometSort (112) - : +- CometColumnarExchange (111) - : +- * HashAggregate (110) - : +- * HashAggregate (109) - : +- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- * Project (92) - : : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : : :- * Project (89) - : : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : : :- * Project (82) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : : :- * Project (79) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : : :- * Project (73) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) - : : : : : : : : :- * Project (67) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : : : : : : : :- * Project (64) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) - : : : : : : : : : : : : : : : : : +- 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) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (26) - : : : : : : : : : : : : : : : : +- CometExchange (25) - : : : : : : : : : : : : : : : : +- CometHashAggregate (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * CometColumnarToRow (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * CometColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * CometColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (62) - : : : : : : : : : : +- * CometColumnarToRow (61) - : : : : : : : : : : +- CometProject (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (65) - : : : : : : : : +- BroadcastExchange (71) - : : : : : : : : +- * CometColumnarToRow (70) - : : : : : : : : +- CometFilter (69) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (68) - : : : : : : : +- BroadcastExchange (77) - : : : : : : : +- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (74) - : : : : : : +- ReusedExchange (80) - : : : : : +- BroadcastExchange (87) - : : : : : +- * CometColumnarToRow (86) - : : : : : +- CometProject (85) - : : : : : +- CometFilter (84) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (83) - : : : : +- ReusedExchange (90) - : : : +- BroadcastExchange (96) - : : : +- * CometColumnarToRow (95) - : : : +- CometFilter (94) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (93) - : : +- ReusedExchange (99) - : +- BroadcastExchange (106) - : +- * CometColumnarToRow (105) - : +- CometProject (104) - : +- CometFilter (103) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (102) - +- CometSort (180) - +- CometColumnarExchange (179) - +- * HashAggregate (178) - +- * HashAggregate (177) - +- * Project (176) - +- * BroadcastHashJoin Inner BuildRight (175) - :- * Project (173) - : +- * BroadcastHashJoin Inner BuildRight (172) - : :- * Project (170) - : : +- * BroadcastHashJoin Inner BuildRight (169) - : : :- * Project (167) - : : : +- * BroadcastHashJoin Inner BuildRight (166) - : : : :- * Project (164) - : : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : : :- * Project (161) - : : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : : :- * Project (158) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : : :- * Project (155) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : : :- * Project (152) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : : :- * 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) - : : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (130) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (123) - : : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : : +- CometExchange (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (119) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (115) - : : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (113) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometFilter (117) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (116) - : : : : : : : : : : : : : : : +- * Sort (129) - : : : : : : : : : : : : : : : +- * Project (128) - : : : : : : : : : : : : : : : +- * Filter (127) - : : : : : : : : : : : : : : : +- * HashAggregate (126) - : : : : : : : : : : : : : : : +- * CometColumnarToRow (125) - : : : : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : : +- ReusedExchange (147) - : : : : : : : : +- ReusedExchange (150) - : : : : : : : +- ReusedExchange (153) - : : : : : : +- ReusedExchange (156) - : : : : : +- ReusedExchange (159) - : : : : +- ReusedExchange (162) - : : : +- ReusedExchange (165) - : : +- ReusedExchange (168) - : +- ReusedExchange (171) - +- ReusedExchange (174) +* 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 @@ -234,846 +225,787 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) CometColumnarToRow [codegen id : 1] -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] - -(12) 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 -(13) 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)) -(14) 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] -(15) 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=2] -(16) 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] -(17) 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 -(18) 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)) -(19) 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] -(20) 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=3] -(21) 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] -(22) 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 -(23) 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] -(24) 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))] -(25) 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=4] -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] - -(27) HashAggregate [codegen id : 2] +(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))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#30, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#30,17,2) AS sale#32, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31 AS refund#33] -(28) Filter [codegen id : 2] -Input [3]: [cs_item_sk#17, sale#32, refund#33] -Condition : ((isnotnull(sale#32) AND isnotnull(refund#33)) AND (cast(sale#32 as decimal(21,2)) > (2 * refund#33))) +(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) Project [codegen id : 2] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#32, refund#33] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) Sort [codegen id : 2] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) SortMergeJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(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) 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, ss_sold_date_sk#12] +(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) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#34, d_year#35] +(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)) -(34) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] -(35) 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#35] -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#34, d_year#35] +(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] (36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#36, s_store_name#37, s_zip#38] +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 -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Condition : ((isnotnull(s_store_sk#36) AND isnotnull(s_store_name#37)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)))) +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(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) (38) CometProject -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Arguments: [s_store_sk#36, s_store_name#37, s_zip#39], [s_store_sk#36, s_store_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)) AS s_zip#39] - -(39) CometColumnarToRow [codegen id : 4] -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] +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, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] -(40) BroadcastExchange -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#36] -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 -(42) 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#35, s_store_name#37, s_zip#39] -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#35, s_store_sk#36, s_store_name#37, s_zip#39] +(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] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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 -(44) CometFilter -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Condition : (((((isnotnull(c_customer_sk#40) AND isnotnull(c_first_sales_date_sk#45)) AND isnotnull(c_first_shipto_date_sk#44)) AND isnotnull(c_current_cdemo_sk#41)) AND isnotnull(c_current_hdemo_sk#42)) AND isnotnull(c_current_addr_sk#43)) - -(45) CometColumnarToRow [codegen id : 5] -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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) BroadcastExchange -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#40] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -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#35, s_store_name#37, s_zip#39, c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] +(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 -(50) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : isnotnull(d_date_sk#46) - -(51) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#46, d_year#47] +(48) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : isnotnull(d_date_sk#44) -(52) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(49) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] -(53) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#45] -Right keys [1]: [d_date_sk#46] -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 -(54) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45, d_date_sk#46, d_year#47] +(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] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#48, d_year#49] +(52) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#46, d_year#47] -(56) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#44] -Right keys [1]: [d_date_sk#48] -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 -(57) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, d_year#47, d_date_sk#48, d_year#49] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#50, cd_marital_status#51] +(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 -(59) CometFilter -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Condition : (isnotnull(cd_demo_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)))) +(56) CometFilter +Input [2]: [cd_demo_sk#48, cd_marital_status#49] +Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) -(60) CometProject -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Arguments: [cd_demo_sk#50, cd_marital_status#52], [cd_demo_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)) AS cd_marital_status#52] - -(61) CometColumnarToRow [codegen id : 8] -Input [2]: [cd_demo_sk#50, cd_marital_status#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] -(62) BroadcastExchange -Input [2]: [cd_demo_sk#50, cd_marital_status#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) CometBroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#50] -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 -(64) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_demo_sk#50, cd_marital_status#52] +(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] -(65) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#53, cd_marital_status#54] +(61) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#41] -Right keys [1]: [cd_demo_sk#53] -Join type: Inner -Join condition: NOT (cd_marital_status#52 = cd_marital_status#54) +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52, cd_demo_sk#53, cd_marital_status#54] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#55] +(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 -(69) CometFilter -Input [1]: [p_promo_sk#55] -Condition : isnotnull(p_promo_sk#55) +(65) CometFilter +Input [1]: [p_promo_sk#53] +Condition : isnotnull(p_promo_sk#53) -(70) CometColumnarToRow [codegen id : 10] -Input [1]: [p_promo_sk#55] +(66) CometBroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(71) BroadcastExchange -Input [1]: [p_promo_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#55] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, p_promo_sk#55] +(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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(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 -(75) CometFilter -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Condition : (isnotnull(hd_demo_sk#56) AND isnotnull(hd_income_band_sk#57)) +(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)) -(76) CometColumnarToRow [codegen id : 11] -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(71) CometBroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(77) BroadcastExchange -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#56] -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 -(79) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_demo_sk#56, hd_income_band_sk#57] +(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] -(80) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#58, hd_income_band_sk#59] +(74) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#42] -Right keys [1]: [hd_demo_sk#58] -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 -(82) 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#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_demo_sk#58, hd_income_band_sk#59] +(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] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] +(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 -(84) CometFilter -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Condition : isnotnull(ca_address_sk#60) +(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) -(85) CometProject -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Arguments: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66], [ca_address_sk#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#61, 10)) AS ca_street_number#65, ca_street_name#62, ca_city#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#64, 10)) AS ca_zip#66] - -(86) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] -(87) BroadcastExchange -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(80) CometBroadcastExchange +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] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#60] -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 -(89) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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] -(90) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(91) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#67] -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 -(92) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#72] +(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 -(94) CometFilter -Input [1]: [ib_income_band_sk#72] -Condition : isnotnull(ib_income_band_sk#72) +(87) CometFilter +Input [1]: [ib_income_band_sk#70] +Condition : isnotnull(ib_income_band_sk#70) -(95) CometColumnarToRow [codegen id : 15] -Input [1]: [ib_income_band_sk#72] +(88) CometBroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(96) BroadcastExchange -Input [1]: [ib_income_band_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(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] -(98) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#72] +(91) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#71] -(99) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#73] +(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 -(100) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#59] -Right keys [1]: [ib_income_band_sk#73] -Join type: Inner -Join condition: None +(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] -(101) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#73] - -(102) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] +(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 -(103) CometFilter -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Condition : ((((((isnotnull(i_current_price#75) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#76, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#75 >= 64.00)) AND (i_current_price#75 <= 74.00)) AND (i_current_price#75 >= 65.00)) AND (i_current_price#75 <= 79.00)) AND isnotnull(i_item_sk#74)) - -(104) CometProject -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Arguments: [i_item_sk#74, i_product_name#78], [i_item_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#77, 50)) AS i_product_name#78] +(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 static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) 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) CometColumnarToRow [codegen id : 17] -Input [2]: [i_item_sk#74, i_product_name#78] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] -(106) BroadcastExchange -Input [2]: [i_item_sk#74, i_product_name#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(97) CometBroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#74] -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 -(108) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] +(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] -(109) HashAggregate [codegen id : 18] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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#79, sum#80, sum#81, sum#82] -Results [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -(110) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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)#87, sum(UnscaledValue(ss_wholesale_cost#9))#88, sum(UnscaledValue(ss_list_price#10))#89, sum(UnscaledValue(ss_coupon_amt#11))#90] -Results [17]: [i_product_name#78 AS product_name#91, i_item_sk#74 AS item_sk#92, s_store_name#37 AS store_name#93, s_zip#39 AS store_zip#94, ca_street_number#65 AS b_street_number#95, ca_street_name#62 AS b_streen_name#96, ca_city#63 AS b_city#97, ca_zip#66 AS b_zip#98, ca_street_number#68 AS c_street_number#99, ca_street_name#69 AS c_street_name#100, ca_city#70 AS c_city#101, ca_zip#71 AS c_zip#102, d_year#35 AS syear#103, count(1)#87 AS cnt#104, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#88,17,2) AS s1#105, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#89,17,2) AS s2#106, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#90,17,2) AS s3#107] -(111) CometColumnarExchange -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: hashpartitioning(item_sk#92, store_name#93, store_zip#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(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) CometSort -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107], [item_sk#92 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, store_zip#94 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] -(113) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] +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 -(114) CometFilter -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Condition : (((((((isnotnull(ss_item_sk#108) AND isnotnull(ss_ticket_number#115)) AND isnotnull(ss_store_sk#113)) AND isnotnull(ss_customer_sk#109)) AND isnotnull(ss_cdemo_sk#110)) AND isnotnull(ss_promo_sk#114)) AND isnotnull(ss_hdemo_sk#111)) AND isnotnull(ss_addr_sk#112)) +(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) CometBroadcastExchange -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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] -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] +(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 -(117) CometFilter -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Condition : (isnotnull(sr_item_sk#121) AND isnotnull(sr_ticket_number#122)) +(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))] -(118) CometProject -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Arguments: [sr_item_sk#121, sr_ticket_number#122], [sr_item_sk#121, sr_ticket_number#122] +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#122, refund#123] +Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) -(119) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Right output [2]: [sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_ticket_number#115], [sr_item_sk#121, sr_ticket_number#122], Inner, BuildLeft +(117) CometProject +Input [3]: [cs_item_sk#114, sale#122, refund#123] +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 [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(121) CometExchange -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: hashpartitioning(ss_item_sk#108, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(122) CometSort -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108 ASC NULLS FIRST] - -(123) CometColumnarToRow [codegen id : 19] -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(124) ReusedExchange [Reuses operator id: 25] -Output [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(125) CometColumnarToRow [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(126) HashAggregate [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#124] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#128)), sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#128))#30, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31] -Results [3]: [cs_item_sk#124, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#128))#30,17,2) AS sale#132, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31 AS refund#133] - -(127) Filter [codegen id : 20] -Input [3]: [cs_item_sk#124, sale#132, refund#133] -Condition : ((isnotnull(sale#132) AND isnotnull(refund#133)) AND (cast(sale#132 as decimal(21,2)) > (2 * refund#133))) - -(128) Project [codegen id : 20] -Output [1]: [cs_item_sk#124] -Input [3]: [cs_item_sk#124, sale#132, refund#133] - -(129) Sort [codegen id : 20] -Input [1]: [cs_item_sk#124] -Arguments: [cs_item_sk#124 ASC NULLS FIRST], false, 0 - -(130) SortMergeJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [cs_item_sk#124] -Join type: Inner -Join condition: None - -(131) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, cs_item_sk#124] - -(132) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#134, d_year#135] - -(133) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#119] -Right keys [1]: [d_date_sk#134] -Join type: Inner -Join condition: None - -(134) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135] -Input [13]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, d_date_sk#134, d_year#135] - -(135) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#136, s_store_name#137, s_zip#138] - -(136) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#113] -Right keys [1]: [s_store_sk#136] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 36] -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138] -Input [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_sk#136, s_store_name#137, s_zip#138] - -(138) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(139) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#109] -Right keys [1]: [c_customer_sk#139] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] -Input [18]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#145, d_year#146] - -(142) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#144] -Right keys [1]: [d_date_sk#145] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144, d_date_sk#145, d_year#146] - -(144) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#147, d_year#148] - -(145) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#143] -Right keys [1]: [d_date_sk#147] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146, d_date_sk#147, d_year#148] - -(147) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#149, cd_marital_status#150] - -(148) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#110] -Right keys [1]: [cd_demo_sk#149] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_demo_sk#149, cd_marital_status#150] - -(150) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#151, cd_marital_status#152] - -(151) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#140] -Right keys [1]: [cd_demo_sk#151] -Join type: Inner -Join condition: NOT (cd_marital_status#150 = cd_marital_status#152) - -(152) Project [codegen id : 36] -Output [14]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150, cd_demo_sk#151, cd_marital_status#152] - -(153) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#153] - -(154) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#114] -Right keys [1]: [p_promo_sk#153] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, p_promo_sk#153] - -(156) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] - -(157) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#111] -Right keys [1]: [hd_demo_sk#154] -Join type: Inner -Join condition: None - -(158) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_demo_sk#154, hd_income_band_sk#155] - -(159) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#156, hd_income_band_sk#157] - -(160) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#141] -Right keys [1]: [hd_demo_sk#156] -Join type: Inner -Join condition: None - -(161) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157] -Input [15]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_demo_sk#156, hd_income_band_sk#157] - -(162) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(163) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#112] -Right keys [1]: [ca_address_sk#158] -Join type: Inner -Join condition: None - -(164) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] -Input [18]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(165) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(166) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#142] -Right keys [1]: [ca_address_sk#163] -Join type: Inner -Join condition: None - -(167) Project [codegen id : 36] -Output [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [21]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(168) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#168] - -(169) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#155] -Right keys [1]: [ib_income_band_sk#168] -Join type: Inner -Join condition: None - -(170) Project [codegen id : 36] -Output [18]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [20]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#168] - -(171) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#169] - -(172) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#157] -Right keys [1]: [ib_income_band_sk#169] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 36] -Output [17]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#169] - -(174) ReusedExchange [Reuses operator id: 106] -Output [2]: [i_item_sk#170, i_product_name#171] - -(175) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [i_item_sk#170] -Join type: Inner -Join condition: None - -(176) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] - -(177) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#116)), partial_sum(UnscaledValue(ss_list_price#117)), partial_sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count#79, sum#172, sum#173, sum#174] -Results [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] - -(178) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#116)), sum(UnscaledValue(ss_list_price#117)), sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count(1)#87, sum(UnscaledValue(ss_wholesale_cost#116))#88, sum(UnscaledValue(ss_list_price#117))#89, sum(UnscaledValue(ss_coupon_amt#118))#90] -Results [8]: [i_item_sk#170 AS item_sk#178, s_store_name#137 AS store_name#179, s_zip#138 AS store_zip#180, d_year#135 AS syear#181, count(1)#87 AS cnt#182, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#116))#88,17,2) AS s1#183, MakeDecimal(sum(UnscaledValue(ss_list_price#117))#89,17,2) AS s2#184, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#118))#90,17,2) AS s3#185] - -(179) CometColumnarExchange -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: hashpartitioning(item_sk#178, store_name#179, store_zip#180, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(180) CometSort -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185], [item_sk#178 ASC NULLS FIRST, store_name#179 ASC NULLS FIRST, store_zip#180 ASC NULLS FIRST] - -(181) CometSortMergeJoin -Left output [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Right output [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#179, store_zip#180], Inner, (cnt#182 <= cnt#104) - -(182) CometProject -Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] - -(183) CometExchange -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] - -(184) CometSort -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST] - -(185) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] +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#124, d_year#125] +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#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) + +(123) CometBroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +Arguments: [d_date_sk#124, d_year#125] + +(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#124, d_year#125] +Arguments: [ss_sold_date_sk#109], [d_date_sk#124], 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#124, d_year#125] +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#125], [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#125] + +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] + +(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#125] +Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] +Arguments: [ss_store_sk#103], [s_store_sk#126], 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#125, s_store_sk#126, s_store_name#127, s_zip#128] +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#125, s_store_name#127, s_zip#128], [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#125, s_store_name#127, s_zip#128] + +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(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#125, s_store_name#127, s_zip#128] +Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Arguments: [ss_customer_sk#99], [c_customer_sk#129], 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#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#135, d_year#136] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Right output [2]: [d_date_sk#135, d_year#136] +Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#137, d_year#138] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] +Right output [2]: [d_date_sk#137, d_year#138] +Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#141, cd_marital_status#142] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] +Right output [2]: [cd_demo_sk#141, cd_marital_status#142] +Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#143] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [1]: [p_promo_sk#143] +Arguments: [ss_promo_sk#104], [p_promo_sk#143], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] +Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] +Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_addr_sk#102], [ca_address_sk#148], 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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#158] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#158] +Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#159] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#159] +Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#160, i_product_name#161] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [2]: [i_item_sk#160, i_product_name#161] +Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +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#170, s2#171, s3#172, syear#168, cnt#169], [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#170, s2#171, s3#172, syear#168, cnt#169] + +(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#170, s2#171, s3#172, syear#168, cnt#169] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 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#170, s2#171, s3#172, syear#168, cnt#169] +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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * CometColumnarToRow (188) - +- CometFilter (187) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (186) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(186) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +(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 -(187) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 1999)) AND isnotnull(d_date_sk#34)) +(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)) -(188) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(179) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(189) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 113 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#120 -BroadcastExchange (193) -+- * CometColumnarToRow (192) - +- CometFilter (191) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (190) +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) -(190) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#134, d_year#135] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#124, d_year#125] 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 -(191) CometFilter -Input [2]: [d_date_sk#134, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND isnotnull(d_date_sk#134)) +(182) CometFilter +Input [2]: [d_date_sk#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(192) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#134, d_year#135] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#124, d_year#125] -(193) BroadcastExchange -Input [2]: [d_date_sk#134, d_year#135] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +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-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt index 41817bd434..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/extended.txt @@ -4,278 +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 - : : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#1, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#2))#3,17,2) AS sale#4, sum(((cr_refunded_cash#5 + cr_reversed_charge#6) + cr_store_credit#7))#8 AS refund#9)] - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#10, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#11))#3,17,2) AS sale#12, sum(((cr_refunded_cash#13 + cr_reversed_charge#14) + cr_store_credit#15))#8 AS refund#16)] - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- 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 132 out of 242 eligible operators (54%). Final plan contains 37 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt index 944f6edbde..43527978d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/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,258 +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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] - InputAdapter - WholeStageCodegen (2) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - 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] #19 - 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] #20 - 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] - 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 - WholeStageCodegen (20) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index 44c02a061d..806525a4e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -1,189 +1,180 @@ == Physical Plan == -* CometColumnarToRow (185) -+- CometSort (184) - +- CometExchange (183) - +- CometProject (182) - +- CometSortMergeJoin (181) - :- CometSort (112) - : +- CometColumnarExchange (111) - : +- * HashAggregate (110) - : +- * HashAggregate (109) - : +- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- * Project (92) - : : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : : :- * Project (89) - : : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : : :- * Project (82) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : : :- * Project (79) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : : :- * Project (73) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) - : : : : : : : : :- * Project (67) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : : : : : : : :- * Project (64) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) - : : : : : : : : : : : : : : : : : +- 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) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (26) - : : : : : : : : : : : : : : : : +- CometExchange (25) - : : : : : : : : : : : : : : : : +- CometHashAggregate (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * CometColumnarToRow (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * CometColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * CometColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (62) - : : : : : : : : : : +- * CometColumnarToRow (61) - : : : : : : : : : : +- CometProject (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (65) - : : : : : : : : +- BroadcastExchange (71) - : : : : : : : : +- * CometColumnarToRow (70) - : : : : : : : : +- CometFilter (69) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (68) - : : : : : : : +- BroadcastExchange (77) - : : : : : : : +- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (74) - : : : : : : +- ReusedExchange (80) - : : : : : +- BroadcastExchange (87) - : : : : : +- * CometColumnarToRow (86) - : : : : : +- CometProject (85) - : : : : : +- CometFilter (84) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (83) - : : : : +- ReusedExchange (90) - : : : +- BroadcastExchange (96) - : : : +- * CometColumnarToRow (95) - : : : +- CometFilter (94) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (93) - : : +- ReusedExchange (99) - : +- BroadcastExchange (106) - : +- * CometColumnarToRow (105) - : +- CometProject (104) - : +- CometFilter (103) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (102) - +- CometSort (180) - +- CometColumnarExchange (179) - +- * HashAggregate (178) - +- * HashAggregate (177) - +- * Project (176) - +- * BroadcastHashJoin Inner BuildRight (175) - :- * Project (173) - : +- * BroadcastHashJoin Inner BuildRight (172) - : :- * Project (170) - : : +- * BroadcastHashJoin Inner BuildRight (169) - : : :- * Project (167) - : : : +- * BroadcastHashJoin Inner BuildRight (166) - : : : :- * Project (164) - : : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : : :- * Project (161) - : : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : : :- * Project (158) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : : :- * Project (155) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : : :- * Project (152) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : : :- * 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) - : : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (130) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (123) - : : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : : +- CometExchange (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (119) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (115) - : : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (113) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometFilter (117) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (116) - : : : : : : : : : : : : : : : +- * Sort (129) - : : : : : : : : : : : : : : : +- * Project (128) - : : : : : : : : : : : : : : : +- * Filter (127) - : : : : : : : : : : : : : : : +- * HashAggregate (126) - : : : : : : : : : : : : : : : +- * CometColumnarToRow (125) - : : : : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : : +- ReusedExchange (147) - : : : : : : : : +- ReusedExchange (150) - : : : : : : : +- ReusedExchange (153) - : : : : : : +- ReusedExchange (156) - : : : : : +- ReusedExchange (159) - : : : : +- ReusedExchange (162) - : : : +- ReusedExchange (165) - : : +- ReusedExchange (168) - : +- ReusedExchange (171) - +- ReusedExchange (174) +* 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 @@ -234,846 +225,787 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) CometColumnarToRow [codegen id : 1] -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] - -(12) 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 -(13) 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)) -(14) 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] -(15) 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=2] -(16) 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] -(17) 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 -(18) 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)) -(19) 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] -(20) 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=3] -(21) 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] -(22) 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 -(23) 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] -(24) 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))] -(25) 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=4] -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] - -(27) HashAggregate [codegen id : 2] +(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))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#30, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#30,17,2) AS sale#32, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31 AS refund#33] -(28) Filter [codegen id : 2] -Input [3]: [cs_item_sk#17, sale#32, refund#33] -Condition : ((isnotnull(sale#32) AND isnotnull(refund#33)) AND (cast(sale#32 as decimal(21,2)) > (2 * refund#33))) +(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) Project [codegen id : 2] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#32, refund#33] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) Sort [codegen id : 2] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) SortMergeJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(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) 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, ss_sold_date_sk#12] +(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) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#34, d_year#35] +(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)) -(34) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] -(35) 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#35] -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#34, d_year#35] +(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] (36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#36, s_store_name#37, s_zip#38] +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 -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Condition : ((isnotnull(s_store_sk#36) AND isnotnull(s_store_name#37)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)))) +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(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) (38) CometProject -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Arguments: [s_store_sk#36, s_store_name#37, s_zip#39], [s_store_sk#36, s_store_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)) AS s_zip#39] - -(39) CometColumnarToRow [codegen id : 4] -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] +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, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] -(40) BroadcastExchange -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#36] -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 -(42) 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#35, s_store_name#37, s_zip#39] -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#35, s_store_sk#36, s_store_name#37, s_zip#39] +(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] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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 -(44) CometFilter -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Condition : (((((isnotnull(c_customer_sk#40) AND isnotnull(c_first_sales_date_sk#45)) AND isnotnull(c_first_shipto_date_sk#44)) AND isnotnull(c_current_cdemo_sk#41)) AND isnotnull(c_current_hdemo_sk#42)) AND isnotnull(c_current_addr_sk#43)) - -(45) CometColumnarToRow [codegen id : 5] -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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) BroadcastExchange -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#40] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -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#35, s_store_name#37, s_zip#39, c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] +(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 -(50) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : isnotnull(d_date_sk#46) - -(51) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#46, d_year#47] +(48) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : isnotnull(d_date_sk#44) -(52) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(49) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] -(53) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#45] -Right keys [1]: [d_date_sk#46] -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 -(54) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45, d_date_sk#46, d_year#47] +(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] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#48, d_year#49] +(52) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#46, d_year#47] -(56) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#44] -Right keys [1]: [d_date_sk#48] -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 -(57) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, d_year#47, d_date_sk#48, d_year#49] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#50, cd_marital_status#51] +(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 -(59) CometFilter -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Condition : (isnotnull(cd_demo_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)))) +(56) CometFilter +Input [2]: [cd_demo_sk#48, cd_marital_status#49] +Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) -(60) CometProject -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Arguments: [cd_demo_sk#50, cd_marital_status#52], [cd_demo_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)) AS cd_marital_status#52] - -(61) CometColumnarToRow [codegen id : 8] -Input [2]: [cd_demo_sk#50, cd_marital_status#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] -(62) BroadcastExchange -Input [2]: [cd_demo_sk#50, cd_marital_status#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) CometBroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#50] -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 -(64) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_demo_sk#50, cd_marital_status#52] +(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] -(65) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#53, cd_marital_status#54] +(61) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#41] -Right keys [1]: [cd_demo_sk#53] -Join type: Inner -Join condition: NOT (cd_marital_status#52 = cd_marital_status#54) +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52, cd_demo_sk#53, cd_marital_status#54] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#55] +(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 -(69) CometFilter -Input [1]: [p_promo_sk#55] -Condition : isnotnull(p_promo_sk#55) +(65) CometFilter +Input [1]: [p_promo_sk#53] +Condition : isnotnull(p_promo_sk#53) -(70) CometColumnarToRow [codegen id : 10] -Input [1]: [p_promo_sk#55] +(66) CometBroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(71) BroadcastExchange -Input [1]: [p_promo_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#55] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, p_promo_sk#55] +(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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(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 -(75) CometFilter -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Condition : (isnotnull(hd_demo_sk#56) AND isnotnull(hd_income_band_sk#57)) +(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)) -(76) CometColumnarToRow [codegen id : 11] -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(71) CometBroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(77) BroadcastExchange -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#56] -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 -(79) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_demo_sk#56, hd_income_band_sk#57] +(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] -(80) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#58, hd_income_band_sk#59] +(74) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#42] -Right keys [1]: [hd_demo_sk#58] -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 -(82) 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#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_demo_sk#58, hd_income_band_sk#59] +(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] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] +(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 -(84) CometFilter -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Condition : isnotnull(ca_address_sk#60) +(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) -(85) CometProject -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Arguments: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66], [ca_address_sk#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#61, 10)) AS ca_street_number#65, ca_street_name#62, ca_city#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#64, 10)) AS ca_zip#66] - -(86) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] -(87) BroadcastExchange -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(80) CometBroadcastExchange +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] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#60] -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 -(89) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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] -(90) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(91) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#67] -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 -(92) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#72] +(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 -(94) CometFilter -Input [1]: [ib_income_band_sk#72] -Condition : isnotnull(ib_income_band_sk#72) +(87) CometFilter +Input [1]: [ib_income_band_sk#70] +Condition : isnotnull(ib_income_band_sk#70) -(95) CometColumnarToRow [codegen id : 15] -Input [1]: [ib_income_band_sk#72] +(88) CometBroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(96) BroadcastExchange -Input [1]: [ib_income_band_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(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] -(98) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#72] +(91) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#71] -(99) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#73] +(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 -(100) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#59] -Right keys [1]: [ib_income_band_sk#73] -Join type: Inner -Join condition: None +(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] -(101) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#73] - -(102) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] +(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 -(103) CometFilter -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Condition : ((((((isnotnull(i_current_price#75) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#76, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#75 >= 64.00)) AND (i_current_price#75 <= 74.00)) AND (i_current_price#75 >= 65.00)) AND (i_current_price#75 <= 79.00)) AND isnotnull(i_item_sk#74)) - -(104) CometProject -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Arguments: [i_item_sk#74, i_product_name#78], [i_item_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#77, 50)) AS i_product_name#78] +(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 static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) 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) CometColumnarToRow [codegen id : 17] -Input [2]: [i_item_sk#74, i_product_name#78] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] -(106) BroadcastExchange -Input [2]: [i_item_sk#74, i_product_name#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(97) CometBroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#74] -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 -(108) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] +(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] -(109) HashAggregate [codegen id : 18] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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#79, sum#80, sum#81, sum#82] -Results [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -(110) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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)#87, sum(UnscaledValue(ss_wholesale_cost#9))#88, sum(UnscaledValue(ss_list_price#10))#89, sum(UnscaledValue(ss_coupon_amt#11))#90] -Results [17]: [i_product_name#78 AS product_name#91, i_item_sk#74 AS item_sk#92, s_store_name#37 AS store_name#93, s_zip#39 AS store_zip#94, ca_street_number#65 AS b_street_number#95, ca_street_name#62 AS b_streen_name#96, ca_city#63 AS b_city#97, ca_zip#66 AS b_zip#98, ca_street_number#68 AS c_street_number#99, ca_street_name#69 AS c_street_name#100, ca_city#70 AS c_city#101, ca_zip#71 AS c_zip#102, d_year#35 AS syear#103, count(1)#87 AS cnt#104, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#88,17,2) AS s1#105, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#89,17,2) AS s2#106, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#90,17,2) AS s3#107] -(111) CometColumnarExchange -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: hashpartitioning(item_sk#92, store_name#93, store_zip#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(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) CometSort -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107], [item_sk#92 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, store_zip#94 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] -(113) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] +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 -(114) CometFilter -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Condition : (((((((isnotnull(ss_item_sk#108) AND isnotnull(ss_ticket_number#115)) AND isnotnull(ss_store_sk#113)) AND isnotnull(ss_customer_sk#109)) AND isnotnull(ss_cdemo_sk#110)) AND isnotnull(ss_promo_sk#114)) AND isnotnull(ss_hdemo_sk#111)) AND isnotnull(ss_addr_sk#112)) +(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) CometBroadcastExchange -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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] -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] +(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 -(117) CometFilter -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Condition : (isnotnull(sr_item_sk#121) AND isnotnull(sr_ticket_number#122)) +(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))] -(118) CometProject -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Arguments: [sr_item_sk#121, sr_ticket_number#122], [sr_item_sk#121, sr_ticket_number#122] +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#122, refund#123] +Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) -(119) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Right output [2]: [sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_ticket_number#115], [sr_item_sk#121, sr_ticket_number#122], Inner, BuildLeft +(117) CometProject +Input [3]: [cs_item_sk#114, sale#122, refund#123] +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 [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(121) CometExchange -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: hashpartitioning(ss_item_sk#108, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(122) CometSort -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108 ASC NULLS FIRST] - -(123) CometColumnarToRow [codegen id : 19] -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(124) ReusedExchange [Reuses operator id: 25] -Output [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(125) CometColumnarToRow [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(126) HashAggregate [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#124] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#128)), sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#128))#30, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31] -Results [3]: [cs_item_sk#124, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#128))#30,17,2) AS sale#132, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31 AS refund#133] - -(127) Filter [codegen id : 20] -Input [3]: [cs_item_sk#124, sale#132, refund#133] -Condition : ((isnotnull(sale#132) AND isnotnull(refund#133)) AND (cast(sale#132 as decimal(21,2)) > (2 * refund#133))) - -(128) Project [codegen id : 20] -Output [1]: [cs_item_sk#124] -Input [3]: [cs_item_sk#124, sale#132, refund#133] - -(129) Sort [codegen id : 20] -Input [1]: [cs_item_sk#124] -Arguments: [cs_item_sk#124 ASC NULLS FIRST], false, 0 - -(130) SortMergeJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [cs_item_sk#124] -Join type: Inner -Join condition: None - -(131) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, cs_item_sk#124] - -(132) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#134, d_year#135] - -(133) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#119] -Right keys [1]: [d_date_sk#134] -Join type: Inner -Join condition: None - -(134) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135] -Input [13]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, d_date_sk#134, d_year#135] - -(135) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#136, s_store_name#137, s_zip#138] - -(136) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#113] -Right keys [1]: [s_store_sk#136] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 36] -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138] -Input [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_sk#136, s_store_name#137, s_zip#138] - -(138) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(139) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#109] -Right keys [1]: [c_customer_sk#139] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] -Input [18]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#145, d_year#146] - -(142) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#144] -Right keys [1]: [d_date_sk#145] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144, d_date_sk#145, d_year#146] - -(144) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#147, d_year#148] - -(145) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#143] -Right keys [1]: [d_date_sk#147] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146, d_date_sk#147, d_year#148] - -(147) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#149, cd_marital_status#150] - -(148) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#110] -Right keys [1]: [cd_demo_sk#149] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_demo_sk#149, cd_marital_status#150] - -(150) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#151, cd_marital_status#152] - -(151) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#140] -Right keys [1]: [cd_demo_sk#151] -Join type: Inner -Join condition: NOT (cd_marital_status#150 = cd_marital_status#152) - -(152) Project [codegen id : 36] -Output [14]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150, cd_demo_sk#151, cd_marital_status#152] - -(153) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#153] - -(154) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#114] -Right keys [1]: [p_promo_sk#153] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, p_promo_sk#153] - -(156) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] - -(157) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#111] -Right keys [1]: [hd_demo_sk#154] -Join type: Inner -Join condition: None - -(158) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_demo_sk#154, hd_income_band_sk#155] - -(159) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#156, hd_income_band_sk#157] - -(160) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#141] -Right keys [1]: [hd_demo_sk#156] -Join type: Inner -Join condition: None - -(161) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157] -Input [15]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_demo_sk#156, hd_income_band_sk#157] - -(162) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(163) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#112] -Right keys [1]: [ca_address_sk#158] -Join type: Inner -Join condition: None - -(164) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] -Input [18]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(165) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(166) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#142] -Right keys [1]: [ca_address_sk#163] -Join type: Inner -Join condition: None - -(167) Project [codegen id : 36] -Output [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [21]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(168) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#168] - -(169) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#155] -Right keys [1]: [ib_income_band_sk#168] -Join type: Inner -Join condition: None - -(170) Project [codegen id : 36] -Output [18]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [20]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#168] - -(171) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#169] - -(172) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#157] -Right keys [1]: [ib_income_band_sk#169] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 36] -Output [17]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#169] - -(174) ReusedExchange [Reuses operator id: 106] -Output [2]: [i_item_sk#170, i_product_name#171] - -(175) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [i_item_sk#170] -Join type: Inner -Join condition: None - -(176) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] - -(177) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#116)), partial_sum(UnscaledValue(ss_list_price#117)), partial_sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count#79, sum#172, sum#173, sum#174] -Results [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] - -(178) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#116)), sum(UnscaledValue(ss_list_price#117)), sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count(1)#87, sum(UnscaledValue(ss_wholesale_cost#116))#88, sum(UnscaledValue(ss_list_price#117))#89, sum(UnscaledValue(ss_coupon_amt#118))#90] -Results [8]: [i_item_sk#170 AS item_sk#178, s_store_name#137 AS store_name#179, s_zip#138 AS store_zip#180, d_year#135 AS syear#181, count(1)#87 AS cnt#182, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#116))#88,17,2) AS s1#183, MakeDecimal(sum(UnscaledValue(ss_list_price#117))#89,17,2) AS s2#184, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#118))#90,17,2) AS s3#185] - -(179) CometColumnarExchange -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: hashpartitioning(item_sk#178, store_name#179, store_zip#180, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(180) CometSort -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185], [item_sk#178 ASC NULLS FIRST, store_name#179 ASC NULLS FIRST, store_zip#180 ASC NULLS FIRST] - -(181) CometSortMergeJoin -Left output [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Right output [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#179, store_zip#180], Inner, (cnt#182 <= cnt#104) - -(182) CometProject -Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] - -(183) CometExchange -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] - -(184) CometSort -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST] - -(185) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] +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#124, d_year#125] +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#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) + +(123) CometBroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +Arguments: [d_date_sk#124, d_year#125] + +(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#124, d_year#125] +Arguments: [ss_sold_date_sk#109], [d_date_sk#124], 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#124, d_year#125] +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#125], [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#125] + +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] + +(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#125] +Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] +Arguments: [ss_store_sk#103], [s_store_sk#126], 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#125, s_store_sk#126, s_store_name#127, s_zip#128] +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#125, s_store_name#127, s_zip#128], [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#125, s_store_name#127, s_zip#128] + +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(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#125, s_store_name#127, s_zip#128] +Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Arguments: [ss_customer_sk#99], [c_customer_sk#129], 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#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#135, d_year#136] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Right output [2]: [d_date_sk#135, d_year#136] +Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#137, d_year#138] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] +Right output [2]: [d_date_sk#137, d_year#138] +Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#141, cd_marital_status#142] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] +Right output [2]: [cd_demo_sk#141, cd_marital_status#142] +Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#143] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [1]: [p_promo_sk#143] +Arguments: [ss_promo_sk#104], [p_promo_sk#143], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] +Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] +Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_addr_sk#102], [ca_address_sk#148], 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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#158] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#158] +Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#159] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#159] +Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#160, i_product_name#161] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [2]: [i_item_sk#160, i_product_name#161] +Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +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#170, s2#171, s3#172, syear#168, cnt#169], [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#170, s2#171, s3#172, syear#168, cnt#169] + +(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#170, s2#171, s3#172, syear#168, cnt#169] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 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#170, s2#171, s3#172, syear#168, cnt#169] +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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * CometColumnarToRow (188) - +- CometFilter (187) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (186) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(186) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +(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 -(187) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 1999)) AND isnotnull(d_date_sk#34)) +(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)) -(188) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(179) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(189) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 113 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#120 -BroadcastExchange (193) -+- * CometColumnarToRow (192) - +- CometFilter (191) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (190) +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) -(190) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#134, d_year#135] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#124, d_year#125] 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 -(191) CometFilter -Input [2]: [d_date_sk#134, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND isnotnull(d_date_sk#134)) +(182) CometFilter +Input [2]: [d_date_sk#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(192) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#134, d_year#135] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#124, d_year#125] -(193) BroadcastExchange -Input [2]: [d_date_sk#134, d_year#135] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +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-v1_4-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt index 41817bd434..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/extended.txt @@ -4,278 +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 - : : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#1, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#2))#3,17,2) AS sale#4, sum(((cr_refunded_cash#5 + cr_reversed_charge#6) + cr_store_credit#7))#8 AS refund#9)] - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#10, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#11))#3,17,2) AS sale#12, sum(((cr_refunded_cash#13 + cr_reversed_charge#14) + cr_store_credit#15))#8 AS refund#16)] - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- 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 132 out of 242 eligible operators (54%). Final plan contains 37 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index 944f6edbde..43527978d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -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,258 +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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] - InputAdapter - WholeStageCodegen (2) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - 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] #19 - 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] #20 - 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] - 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 - WholeStageCodegen (20) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt index 0ce02445c2..6dc60931a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/explain.txt @@ -1,48 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (17) - : : +- * Filter (16) - : : +- * HashAggregate (15) - : : +- * CometColumnarToRow (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (41) - +- * Filter (40) - +- * HashAggregate (39) - +- * CometColumnarToRow (38) - +- CometColumnarExchange (37) - +- * HashAggregate (36) - +- * HashAggregate (35) - +- * CometColumnarToRow (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (28) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (27) - +- ReusedExchange (29) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometHashAggregate (13) + : : +- CometExchange (12) + : : +- CometHashAggregate (11) + : : +- CometProject (10) + : : +- CometBroadcastHashJoin (9) + : : :- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : +- CometBroadcastExchange (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) + : +- CometBroadcastExchange (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) + +- CometBroadcastExchange (36) + +- CometFilter (35) + +- CometHashAggregate (34) + +- CometExchange (33) + +- CometHashAggregate (32) + +- CometHashAggregate (31) + +- CometExchange (30) + +- CometHashAggregate (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) + +- ReusedExchange (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store @@ -56,10 +52,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) CometColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -67,225 +60,205 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [2]: [d_date_sk#8, d_month_seq#9] Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(8) CometProject +(7) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [1]: [d_date_sk#8] Arguments: [d_date_sk#8] -(10) CometBroadcastHashJoin +(9) CometBroadcastHashJoin Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Right output [1]: [d_date_sk#8] Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight -(11) CometProject +(10) CometProject Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(12) CometHashAggregate +(11) CometHashAggregate Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -(13) CometExchange +(12) CometExchange Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(14) CometColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(15) HashAggregate [codegen id : 1] +(13) CometHashAggregate Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(16) Filter [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(14) CometFilter +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Condition : isnotnull(revenue#11) -(17) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(15) CometBroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#4] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [s_store_sk#1, s_store_name#2] +Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight -(19) Project [codegen id : 5] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +(17) CometProject +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(21) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) +(19) CometFilter +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Condition : isnotnull(i_item_sk#12) -(22) CometProject -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#17, 50)) AS i_brand#18] +(20) CometProject +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#17] -(23) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] +(21) CometBroadcastExchange +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -(24) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(23) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -(26) Project [codegen id : 5] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_store_sk#20) +(25) CometFilter +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_store_sk#19) + +(26) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#23] + +(27) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(28) CometProject +Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] +Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -(29) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#24] +(29) CometHashAggregate +Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -(30) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Right output [1]: [d_date_sk#24] -Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight +(30) CometExchange +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] +Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(31) CometProject -Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24] -Arguments: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21], [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] +(31) CometHashAggregate +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] (32) CometHashAggregate -Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] +Input [2]: [ss_store_sk#19, revenue#25] +Keys [1]: [ss_store_sk#19] +Functions [1]: [partial_avg(revenue#25)] (33) CometExchange -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(35) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] -Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] - -(36) HashAggregate [codegen id : 3] -Input [2]: [ss_store_sk#20, revenue#27] -Keys [1]: [ss_store_sk#20] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#20, sum#30, count#31] - -(37) CometColumnarExchange -Input [3]: [ss_store_sk#20, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#20, sum#30, count#31] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_store_sk#20, sum#30, count#31] -Keys [1]: [ss_store_sk#20] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] - -(40) Filter [codegen id : 4] -Input [2]: [ss_store_sk#20, ave#33] -Condition : isnotnull(ave#33) - -(41) BroadcastExchange -Input [2]: [ss_store_sk#20, ave#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#20] -Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) - -(43) Project [codegen id : 5] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] - -(44) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] +Input [3]: [ss_store_sk#19, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(34) CometHashAggregate +Input [3]: [ss_store_sk#19, sum#26, count#27] +Keys [1]: [ss_store_sk#19] +Functions [1]: [avg(revenue#25)] + +(35) CometFilter +Input [2]: [ss_store_sk#19, ave#28] +Condition : isnotnull(ave#28) + +(36) CometBroadcastExchange +Input [2]: [ss_store_sk#19, ave#28] +Arguments: [ss_store_sk#19, ave#28] + +(37) CometBroadcastHashJoin +Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Right output [2]: [ss_store_sk#19, ave#28] +Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight + +(38) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] +Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] + +(39) CometTakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] + +(40) CometColumnarToRow [codegen id : 1] +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +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 +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(42) CometFilter Input [2]: [d_date_sk#8, d_month_seq#9] Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(47) CometProject +(43) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(48) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(49) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt index d23062c2b5..1fc09bd2e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_store_sk#1, ss_item_sk#2, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#4,17,2) AS revenue#5)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- 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 - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_store_sk#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS revenue#9)] - +- CometColumnarToRow - +- 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 + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- 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 + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 29 out of 48 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt index 8f3fe39fef..cd1c84b6f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65.native_iceberg_compat/simplified.txt @@ -1,66 +1,51 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (5) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] + CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] + CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] CometFilter [s_store_sk,s_store_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (1) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - 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 #3 - 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] #4 - 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 #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #7 - WholeStageCodegen (3) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - 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] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 + CometFilter [ss_store_sk,ss_item_sk,revenue] + CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_store_sk,ss_item_sk] #2 + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + 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 #3 + 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] #4 + 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_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 + CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastExchange [ss_store_sk,ave] #6 + CometFilter [ss_store_sk,ave] + CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] + CometExchange [ss_store_sk] #7 + CometHashAggregate [revenue] [ss_store_sk,sum,count] + CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_store_sk,ss_item_sk] #8 + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + 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] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt index 0ce02445c2..6dc60931a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt @@ -1,48 +1,44 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * Project (43) - +- * BroadcastHashJoin Inner BuildRight (42) - :- * Project (26) - : +- * BroadcastHashJoin Inner BuildRight (25) - : :- * Project (19) - : : +- * BroadcastHashJoin Inner BuildRight (18) - : : :- * CometColumnarToRow (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) - : : +- BroadcastExchange (17) - : : +- * Filter (16) - : : +- * HashAggregate (15) - : : +- * CometColumnarToRow (14) - : : +- CometExchange (13) - : : +- CometHashAggregate (12) - : : +- CometProject (11) - : : +- CometBroadcastHashJoin (10) - : : :- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : +- CometBroadcastExchange (9) - : : +- CometProject (8) - : : +- CometFilter (7) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (6) - : +- BroadcastExchange (24) - : +- * CometColumnarToRow (23) - : +- CometProject (22) - : +- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (20) - +- BroadcastExchange (41) - +- * Filter (40) - +- * HashAggregate (39) - +- * CometColumnarToRow (38) - +- CometColumnarExchange (37) - +- * HashAggregate (36) - +- * HashAggregate (35) - +- * CometColumnarToRow (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (28) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (27) - +- ReusedExchange (29) +* CometColumnarToRow (40) ++- CometTakeOrderedAndProject (39) + +- CometProject (38) + +- CometBroadcastHashJoin (37) + :- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (1) + : : +- CometBroadcastExchange (15) + : : +- CometFilter (14) + : : +- CometHashAggregate (13) + : : +- CometExchange (12) + : : +- CometHashAggregate (11) + : : +- CometProject (10) + : : +- CometBroadcastHashJoin (9) + : : :- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (3) + : : +- CometBroadcastExchange (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (5) + : +- CometBroadcastExchange (21) + : +- CometProject (20) + : +- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (18) + +- CometBroadcastExchange (36) + +- CometFilter (35) + +- CometHashAggregate (34) + +- CometExchange (33) + +- CometHashAggregate (32) + +- CometHashAggregate (31) + +- CometExchange (30) + +- CometHashAggregate (29) + +- CometProject (28) + +- CometBroadcastHashJoin (27) + :- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (24) + +- ReusedExchange (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store @@ -56,10 +52,7 @@ ReadSchema: struct Input [2]: [s_store_sk#1, s_store_name#2] Condition : isnotnull(s_store_sk#1) -(3) CometColumnarToRow [codegen id : 5] -Input [2]: [s_store_sk#1, s_store_name#2] - -(4) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(3) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Batched: true Location: InMemoryFileIndex [] @@ -67,225 +60,205 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sol PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] ReadSchema: struct -(5) CometFilter +(4) CometFilter Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) -(6) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(5) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(7) CometFilter +(6) CometFilter Input [2]: [d_date_sk#8, d_month_seq#9] Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(8) CometProject +(7) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(9) CometBroadcastExchange +(8) CometBroadcastExchange Input [1]: [d_date_sk#8] Arguments: [d_date_sk#8] -(10) CometBroadcastHashJoin +(9) CometBroadcastHashJoin Left output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] Right output [1]: [d_date_sk#8] Arguments: [ss_sold_date_sk#6], [d_date_sk#8], Inner, BuildRight -(11) CometProject +(10) CometProject Input [5]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6, d_date_sk#8] Arguments: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5], [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] -(12) CometHashAggregate +(11) CometHashAggregate Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] -(13) CometExchange +(12) CometExchange Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(14) CometColumnarToRow [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] - -(15) HashAggregate [codegen id : 1] +(13) CometHashAggregate Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] Keys [2]: [ss_store_sk#4, ss_item_sk#3] Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] -Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] -(16) Filter [codegen id : 1] -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Condition : isnotnull(revenue#12) +(14) CometFilter +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Condition : isnotnull(revenue#11) -(17) BroadcastExchange -Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(15) CometBroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [ss_store_sk#4, ss_item_sk#3, revenue#11] -(18) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [s_store_sk#1] -Right keys [1]: [ss_store_sk#4] -Join type: Inner -Join condition: None +(16) CometBroadcastHashJoin +Left output [2]: [s_store_sk#1, s_store_name#2] +Right output [3]: [ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_sk#1], [ss_store_sk#4], Inner, BuildRight -(19) Project [codegen id : 5] -Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] -Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +(17) CometProject +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Arguments: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11], [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +Output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(21) CometFilter -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Condition : isnotnull(i_item_sk#13) +(19) CometFilter +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Condition : isnotnull(i_item_sk#12) -(22) CometProject -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] -Arguments: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18], [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#17, 50)) AS i_brand#18] +(20) CometProject +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#16] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_brand#16, 50)) AS i_brand#17] -(23) CometColumnarToRow [codegen id : 2] -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] +(21) CometBroadcastExchange +Input [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -(24) BroadcastExchange -Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +(22) CometBroadcastHashJoin +Left output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11] +Right output [5]: [i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [ss_item_sk#3], [i_item_sk#12], Inner, BuildRight -(25) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#3] -Right keys [1]: [i_item_sk#13] -Join type: Inner -Join condition: None +(23) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#11, i_item_sk#12, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] -(26) Project [codegen id : 5] -Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18] - -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_store_sk#20) +(25) CometFilter +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_store_sk#19) + +(26) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#23] + +(27) CometBroadcastHashJoin +Left output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Right output [1]: [d_date_sk#23] +Arguments: [ss_sold_date_sk#21], [d_date_sk#23], Inner, BuildRight + +(28) CometProject +Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] +Arguments: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20], [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] -(29) ReusedExchange [Reuses operator id: 9] -Output [1]: [d_date_sk#24] +(29) CometHashAggregate +Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] -(30) CometBroadcastHashJoin -Left output [4]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Right output [1]: [d_date_sk#24] -Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight +(30) CometExchange +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] +Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(31) CometProject -Input [5]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24] -Arguments: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21], [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] +(31) CometHashAggregate +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#24] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] (32) CometHashAggregate -Input [3]: [ss_item_sk#19, ss_store_sk#20, ss_sales_price#21] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] +Input [2]: [ss_store_sk#19, revenue#25] +Keys [1]: [ss_store_sk#19] +Functions [1]: [partial_avg(revenue#25)] (33) CometExchange -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Arguments: hashpartitioning(ss_store_sk#20, ss_item_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(34) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] - -(35) HashAggregate [codegen id : 3] -Input [3]: [ss_store_sk#20, ss_item_sk#19, sum#25] -Keys [2]: [ss_store_sk#20, ss_item_sk#19] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#21))#26] -Results [2]: [ss_store_sk#20, MakeDecimal(sum(UnscaledValue(ss_sales_price#21))#26,17,2) AS revenue#27] - -(36) HashAggregate [codegen id : 3] -Input [2]: [ss_store_sk#20, revenue#27] -Keys [1]: [ss_store_sk#20] -Functions [1]: [partial_avg(revenue#27)] -Aggregate Attributes [2]: [sum#28, count#29] -Results [3]: [ss_store_sk#20, sum#30, count#31] - -(37) CometColumnarExchange -Input [3]: [ss_store_sk#20, sum#30, count#31] -Arguments: hashpartitioning(ss_store_sk#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(38) CometColumnarToRow [codegen id : 4] -Input [3]: [ss_store_sk#20, sum#30, count#31] - -(39) HashAggregate [codegen id : 4] -Input [3]: [ss_store_sk#20, sum#30, count#31] -Keys [1]: [ss_store_sk#20] -Functions [1]: [avg(revenue#27)] -Aggregate Attributes [1]: [avg(revenue#27)#32] -Results [2]: [ss_store_sk#20, avg(revenue#27)#32 AS ave#33] - -(40) Filter [codegen id : 4] -Input [2]: [ss_store_sk#20, ave#33] -Condition : isnotnull(ave#33) - -(41) BroadcastExchange -Input [2]: [ss_store_sk#20, ave#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] - -(42) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [ss_store_sk#20] -Join type: Inner -Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) - -(43) Project [codegen id : 5] -Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#18, ss_store_sk#20, ave#33] - -(44) TakeOrderedAndProject -Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] -Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#18] +Input [3]: [ss_store_sk#19, sum#26, count#27] +Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(34) CometHashAggregate +Input [3]: [ss_store_sk#19, sum#26, count#27] +Keys [1]: [ss_store_sk#19] +Functions [1]: [avg(revenue#25)] + +(35) CometFilter +Input [2]: [ss_store_sk#19, ave#28] +Condition : isnotnull(ave#28) + +(36) CometBroadcastExchange +Input [2]: [ss_store_sk#19, ave#28] +Arguments: [ss_store_sk#19, ave#28] + +(37) CometBroadcastHashJoin +Left output [7]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Right output [2]: [ss_store_sk#19, ave#28] +Arguments: [ss_store_sk#4], [ss_store_sk#19], Inner, (cast(revenue#11 as decimal(23,7)) <= (0.1 * ave#28)), BuildRight + +(38) CometProject +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#11, i_item_desc#13, i_current_price#14, i_wholesale_cost#15, i_brand#17, ss_store_sk#19, ave#28] +Arguments: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] + +(39) CometTakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#2 ASC NULLS FIRST,i_item_desc#13 ASC NULLS FIRST], output=[s_store_name#2,i_item_desc#13,revenue#11,i_current_price#14,i_wholesale_cost#15,i_brand#17]), [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17], 100, 0, [s_store_name#2 ASC NULLS FIRST, i_item_desc#13 ASC NULLS FIRST], [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] + +(40) CometColumnarToRow [codegen id : 1] +Input [6]: [s_store_name#2, i_item_desc#13, revenue#11, i_current_price#14, i_wholesale_cost#15, i_brand#17] ===== Subqueries ===== -Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +Subquery:1 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +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 +(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#8, d_month_seq#9] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(42) CometFilter Input [2]: [d_date_sk#8, d_month_seq#9] Condition : (((isnotnull(d_month_seq#9) AND (d_month_seq#9 >= 1176)) AND (d_month_seq#9 <= 1187)) AND isnotnull(d_date_sk#8)) -(47) CometProject +(43) CometProject Input [2]: [d_date_sk#8, d_month_seq#9] Arguments: [d_date_sk#8], [d_date_sk#8] -(48) CometColumnarToRow [codegen id : 1] +(44) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#8] -(49) BroadcastExchange +(45) BroadcastExchange Input [1]: [d_date_sk#8] -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=4] -Subquery:2 Hosting operator id = 27 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt index d23062c2b5..1fc09bd2e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/extended.txt @@ -1,56 +1,52 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_store_sk#1, ss_item_sk#2, MakeDecimal(sum(UnscaledValue(ss_sales_price#3))#4,17,2) AS revenue#5)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- 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 - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- Filter - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_store_sk#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS revenue#9)] - +- CometColumnarToRow - +- 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 + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- 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 + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 29 out of 48 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 48 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt index 8f3fe39fef..cd1c84b6f2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -1,66 +1,51 @@ -TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - WholeStageCodegen (5) - Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] - Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Project [s_store_name,ss_store_sk,ss_item_sk,revenue] - BroadcastHashJoin [s_store_sk,ss_store_sk] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand,ss_store_sk,ave] + CometProject [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastHashJoin [s_store_name,ss_store_sk,ss_item_sk,revenue,i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometProject [s_store_name,ss_store_sk,ss_item_sk,revenue] + CometBroadcastHashJoin [s_store_sk,s_store_name,ss_store_sk,ss_item_sk,revenue] CometFilter [s_store_sk,s_store_name] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name] - InputAdapter - BroadcastExchange #1 - WholeStageCodegen (1) - Filter [revenue] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_store_sk,ss_item_sk] #2 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - 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 #3 - 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] #4 - 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 #5 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Filter [ave] - HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ss_store_sk] #7 - WholeStageCodegen (3) - HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] - HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_store_sk,ss_item_sk] #8 - CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] - CometProject [ss_item_sk,ss_store_sk,ss_sales_price] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] - 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] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [ss_store_sk,ss_item_sk,revenue] #1 + CometFilter [ss_store_sk,ss_item_sk,revenue] + CometHashAggregate [sum] [ss_store_sk,ss_item_sk,revenue,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_store_sk,ss_item_sk] #2 + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + 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 #3 + 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] #4 + 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_item_desc,i_current_price,i_wholesale_cost,i_brand] #5 + CometProject [i_brand] [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometFilter [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + CometBroadcastExchange [ss_store_sk,ave] #6 + CometFilter [ss_store_sk,ave] + CometHashAggregate [sum,count] [ss_store_sk,ave,avg(revenue)] + CometExchange [ss_store_sk] #7 + CometHashAggregate [revenue] [ss_store_sk,sum,count] + CometHashAggregate [ss_item_sk,sum] [ss_store_sk,revenue,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_store_sk,ss_item_sk] #8 + CometHashAggregate [ss_sales_price] [ss_store_sk,ss_item_sk,sum] + CometProject [ss_item_sk,ss_store_sk,ss_sales_price] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk] + 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] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt index 435ad7d126..02afc31b71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/explain.txt @@ -1,47 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- 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 (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (30) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (37) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometHashAggregate (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- 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 (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) + +- ReusedExchange (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -173,110 +169,88 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(Un Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 3] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 3] +(28) CometHashAggregate Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#24, sum(UnscaledValue(ss_ext_list_price#7))#25, sum(UnscaledValue(ss_ext_tax#8))#26] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#20 AS bought_city#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#24,17,2) AS extended_price#28, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#25,17,2) AS list_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#26,17,2) AS extended_tax#30] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_current_addr_sk#32)) +(30) CometFilter +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) -(32) CometProject -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36], [c_customer_sk#31, c_current_addr_sk#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#36] +(31) CometProject +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] +(32) CometBroadcastExchange +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -(34) BroadcastExchange -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(33) CometBroadcastHashJoin +Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] +Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None +(34) CometProject +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -(36) Project [codegen id : 3] -Output [8]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#35, c_last_name#36] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] +(35) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#34, ca_city#35] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#37, ca_city#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#37, ca_city#38] -Condition : (isnotnull(ca_address_sk#37) AND isnotnull(ca_city#38)) - -(39) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#37, ca_city#38] +(36) CometBroadcastHashJoin +Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Right output [2]: [ca_address_sk#34, ca_city#35] +Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight -(40) BroadcastExchange -Input [2]: [ca_address_sk#37, ca_city#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(37) CometProject +Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] +Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -(41) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_current_addr_sk#32] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: NOT (ca_city#38 = bought_city#27) +(38) CometTakeOrderedAndProject +Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -(42) Project [codegen id : 3] -Output [8]: [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] -Input [10]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#35, c_last_name#36, ca_address_sk#37, ca_city#38] - -(43) TakeOrderedAndProject -Input [8]: [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] -Arguments: 100, [c_last_name#36 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] +(39) CometColumnarToRow [codegen id : 1] +Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_dom#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(41) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(46) CometProject +(42) CometProject Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(47) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(48) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#11] -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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt index 1a87e0765b..c842000e11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/extended.txt @@ -1,51 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, ca_city#3 AS bought_city#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS extended_price#7, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#8))#9,17,2) AS list_price#10, MakeDecimal(sum(UnscaledValue(ss_ext_tax#11))#12,17,2) AS extended_tax#13)] - : : +- CometColumnarToRow - : : +- 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.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 35 out of 45 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt index 96fde02c0a..fdddd82de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68.native_iceberg_compat/simplified.txt @@ -1,12 +1,12 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (3) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] @@ -42,18 +42,8 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt index 435ad7d126..02afc31b71 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt @@ -1,47 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- * BroadcastHashJoin Inner BuildRight (41) - :- * Project (36) - : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- 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 (23) - : : +- CometFilter (22) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) - : +- BroadcastExchange (34) - : +- * CometColumnarToRow (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (30) - +- BroadcastExchange (40) - +- * CometColumnarToRow (39) - +- CometFilter (38) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (37) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometProject (37) + +- CometBroadcastHashJoin (36) + :- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometHashAggregate (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometProject (25) + : : +- CometBroadcastHashJoin (24) + : : :- 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 (23) + : : +- CometFilter (22) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (21) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometFilter (30) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (29) + +- ReusedExchange (35) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -173,110 +169,88 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(Un Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(28) CometColumnarToRow [codegen id : 3] -Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 3] +(28) CometHashAggregate Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20, sum#21, sum#22, sum#23] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#20] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#24, sum(UnscaledValue(ss_ext_list_price#7))#25, sum(UnscaledValue(ss_ext_tax#8))#26] -Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#20 AS bought_city#27, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#24,17,2) AS extended_price#28, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#25,17,2) AS list_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#26,17,2) AS extended_tax#30] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(31) CometFilter -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] -Condition : (isnotnull(c_customer_sk#31) AND isnotnull(c_current_addr_sk#32)) +(30) CometFilter +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Condition : (isnotnull(c_customer_sk#24) AND isnotnull(c_current_addr_sk#25)) -(32) CometProject -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#33, c_last_name#34] -Arguments: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36], [c_customer_sk#31, c_current_addr_sk#32, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#36] +(31) CometProject +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#26, c_last_name#27] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#24, c_current_addr_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] -(33) CometColumnarToRow [codegen id : 1] -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] +(32) CometBroadcastExchange +Input [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -(34) BroadcastExchange -Input [4]: [c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(33) CometBroadcastHashJoin +Left output [6]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33] +Right output [4]: [c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [ss_customer_sk#1], [c_customer_sk#24], Inner, BuildRight -(35) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#31] -Join type: Inner -Join condition: None +(34) CometProject +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_customer_sk#24, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Arguments: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29], [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] -(36) Project [codegen id : 3] -Output [8]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#35, c_last_name#36] -Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_customer_sk#31, c_current_addr_sk#32, c_first_name#35, c_last_name#36] +(35) ReusedExchange [Reuses operator id: 23] +Output [2]: [ca_address_sk#34, ca_city#35] -(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [2]: [ca_address_sk#37, ca_city#38] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] -ReadSchema: struct - -(38) CometFilter -Input [2]: [ca_address_sk#37, ca_city#38] -Condition : (isnotnull(ca_address_sk#37) AND isnotnull(ca_city#38)) - -(39) CometColumnarToRow [codegen id : 2] -Input [2]: [ca_address_sk#37, ca_city#38] +(36) CometBroadcastHashJoin +Left output [8]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29] +Right output [2]: [ca_address_sk#34, ca_city#35] +Arguments: [c_current_addr_sk#25], [ca_address_sk#34], Inner, NOT (ca_city#35 = bought_city#30), BuildRight -(40) BroadcastExchange -Input [2]: [ca_address_sk#37, ca_city#38] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] +(37) CometProject +Input [10]: [ss_ticket_number#5, bought_city#30, extended_price#31, list_price#32, extended_tax#33, c_current_addr_sk#25, c_first_name#28, c_last_name#29, ca_address_sk#34, ca_city#35] +Arguments: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -(41) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [c_current_addr_sk#32] -Right keys [1]: [ca_address_sk#37] -Join type: Inner -Join condition: NOT (ca_city#38 = bought_city#27) +(38) CometTakeOrderedAndProject +Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#29 ASC NULLS FIRST,ss_ticket_number#5 ASC NULLS FIRST], output=[c_last_name#29,c_first_name#28,ca_city#35,bought_city#30,ss_ticket_number#5,extended_price#31,extended_tax#33,list_price#32]), [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32], 100, 0, [c_last_name#29 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] -(42) Project [codegen id : 3] -Output [8]: [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] -Input [10]: [ss_ticket_number#5, bought_city#27, extended_price#28, list_price#29, extended_tax#30, c_current_addr_sk#32, c_first_name#35, c_last_name#36, ca_address_sk#37, ca_city#38] - -(43) TakeOrderedAndProject -Input [8]: [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] -Arguments: 100, [c_last_name#36 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#36, c_first_name#35, ca_city#38, bought_city#27, ss_ticket_number#5, extended_price#28, extended_tax#30, list_price#29] +(39) CometColumnarToRow [codegen id : 1] +Input [8]: [c_last_name#29, c_first_name#28, ca_city#35, bought_city#30, ss_ticket_number#5, extended_price#31, extended_tax#33, list_price#32] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_dom#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(41) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Condition : ((((isnotnull(d_dom#13) AND (d_dom#13 >= 1)) AND (d_dom#13 <= 2)) AND d_year#12 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) -(46) CometProject +(42) CometProject Input [3]: [d_date_sk#11, d_year#12, d_dom#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(47) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(48) BroadcastExchange +(44) BroadcastExchange Input [1]: [d_date_sk#11] -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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt index 1a87e0765b..c842000e11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/extended.txt @@ -1,51 +1,49 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, ca_city#3 AS bought_city#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS extended_price#7, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#8))#9,17,2) AS list_price#10, MakeDecimal(sum(UnscaledValue(ss_ext_tax#11))#12,17,2) AS extended_tax#13)] - : : +- CometColumnarToRow - : : +- 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.date_dim - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 35 out of 45 eligible operators (77%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 43 out of 45 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt index 96fde02c0a..fdddd82de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -1,12 +1,12 @@ -TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] - WholeStageCodegen (3) - Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] - Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + CometBroadcastHashJoin [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name,ca_address_sk,ca_city] + CometProject [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum,sum] [ss_ticket_number,ss_customer_sk,bought_city,extended_price,list_price,extended_tax,ca_city,sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 CometHashAggregate [ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] @@ -42,18 +42,8 @@ TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_ CometBroadcastExchange [ca_address_sk,ca_city] #6 CometFilter [ca_address_sk,ca_city] CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometFilter [ca_address_sk,ca_city] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + CometBroadcastExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] #7 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + ReusedExchange [ca_address_sk,ca_city] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt index 23cfae9635..44f6ce791c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt @@ -32,9 +32,9 @@ TakeOrderedAndProject (49) +- * Filter (31) +- Window (30) +- WindowGroupLimit (29) - +- * Sort (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometHashAggregate (26) +- CometExchange (25) +- CometHashAggregate (24) +- CometProject (23) @@ -168,35 +168,33 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometColumnarToRow [codegen id : 1] -Input [2]: [s_state#16, sum#18] - -(27) HashAggregate [codegen id : 1] +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#16] -(28) Sort [codegen id : 1] -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(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] + +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] (29) WindowGroupLimit -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final +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#20, s_state#16] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#16, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#16], [_w0#20 DESC NULLS LAST] +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#20, s_state#16, ranking#21] -Condition : (ranking#21 <= 5) +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#20, s_state#16, ranking#21] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] (33) BroadcastExchange Input [1]: [s_state#16] @@ -209,11 +207,11 @@ Join type: LeftSemi Join condition: None (35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#22] +Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#22] +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] (37) BroadcastHashJoin [codegen id : 4] @@ -223,56 +221,56 @@ Join type: Inner Join condition: None (38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#22, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#22] +Output [3]: [ss_net_profit#2, s_state#21, s_county#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] (39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#22, s_county#8] -Arguments: [[ss_net_profit#2, s_state#22, s_county#8, 0], [ss_net_profit#2, s_state#22, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#23, s_county#24, spark_grouping_id#25] +Input [3]: [ss_net_profit#2, s_state#21, s_county#8] +Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] (40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#23, s_county#24, spark_grouping_id#25] -Keys [3]: [s_state#23, s_county#24, spark_grouping_id#25] +Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#26] -Results [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] +Aggregate Attributes [1]: [sum#25] +Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (41) CometColumnarExchange -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] -Arguments: hashpartitioning(s_state#23, s_county#24, spark_grouping_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (43) HashAggregate [codegen id : 5] -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] -Keys [3]: [s_state#23, s_county#24, spark_grouping_id#25] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#23, s_county#24, (cast((shiftright(spark_grouping_id#25, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint)) AS lochierarchy#30, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w0#31, (cast((shiftright(spark_grouping_id#25, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint)) AS _w1#32, CASE WHEN (cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint) = 0) THEN s_state#23 END AS _w2#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] (44) CometColumnarExchange -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (45) CometSort -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] (46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] (47) Window -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] (48) Project [codegen id : 7] -Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] +Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] (49) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt index 90bcb79a34..45a2c7a669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt @@ -37,9 +37,9 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt index f6953d47ff..80f0cfc8c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt @@ -54,10 +54,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #7 CometHashAggregate [ss_net_profit] [s_state,sum] CometProject [ss_net_profit,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt index 23cfae9635..44f6ce791c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -32,9 +32,9 @@ TakeOrderedAndProject (49) +- * Filter (31) +- Window (30) +- WindowGroupLimit (29) - +- * Sort (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometHashAggregate (26) +- CometExchange (25) +- CometHashAggregate (24) +- CometProject (23) @@ -168,35 +168,33 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometColumnarToRow [codegen id : 1] -Input [2]: [s_state#16, sum#18] - -(27) HashAggregate [codegen id : 1] +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#16] -(28) Sort [codegen id : 1] -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(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] + +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] (29) WindowGroupLimit -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final +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#20, s_state#16] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#16, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#16], [_w0#20 DESC NULLS LAST] +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#20, s_state#16, ranking#21] -Condition : (ranking#21 <= 5) +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#20, s_state#16, ranking#21] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] (33) BroadcastExchange Input [1]: [s_state#16] @@ -209,11 +207,11 @@ Join type: LeftSemi Join condition: None (35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#22] +Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#22] +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] (37) BroadcastHashJoin [codegen id : 4] @@ -223,56 +221,56 @@ Join type: Inner Join condition: None (38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#22, s_county#8] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#22] +Output [3]: [ss_net_profit#2, s_state#21, s_county#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] (39) Expand [codegen id : 4] -Input [3]: [ss_net_profit#2, s_state#22, s_county#8] -Arguments: [[ss_net_profit#2, s_state#22, s_county#8, 0], [ss_net_profit#2, s_state#22, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#23, s_county#24, spark_grouping_id#25] +Input [3]: [ss_net_profit#2, s_state#21, s_county#8] +Arguments: [[ss_net_profit#2, s_state#21, s_county#8, 0], [ss_net_profit#2, s_state#21, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] (40) HashAggregate [codegen id : 4] -Input [4]: [ss_net_profit#2, s_state#23, s_county#24, spark_grouping_id#25] -Keys [3]: [s_state#23, s_county#24, spark_grouping_id#25] +Input [4]: [ss_net_profit#2, s_state#22, s_county#23, spark_grouping_id#24] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#26] -Results [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] +Aggregate Attributes [1]: [sum#25] +Results [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (41) CometColumnarExchange -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] -Arguments: hashpartitioning(s_state#23, s_county#24, spark_grouping_id#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Arguments: hashpartitioning(s_state#22, s_county#23, spark_grouping_id#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] (42) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] (43) HashAggregate [codegen id : 5] -Input [4]: [s_state#23, s_county#24, spark_grouping_id#25, sum#27] -Keys [3]: [s_state#23, s_county#24, spark_grouping_id#25] +Input [4]: [s_state#22, s_county#23, spark_grouping_id#24, sum#26] +Keys [3]: [s_state#22, s_county#23, spark_grouping_id#24] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#28] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS total_sum#29, s_state#23, s_county#24, (cast((shiftright(spark_grouping_id#25, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint)) AS lochierarchy#30, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#28,17,2) AS _w0#31, (cast((shiftright(spark_grouping_id#25, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint)) AS _w1#32, CASE WHEN (cast((shiftright(spark_grouping_id#25, 0) & 1) as tinyint) = 0) THEN s_state#23 END AS _w2#33] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#27] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS total_sum#28, s_state#22, s_county#23, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS lochierarchy#29, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#27,17,2) AS _w0#30, (cast((shiftright(spark_grouping_id#24, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint)) AS _w1#31, CASE WHEN (cast((shiftright(spark_grouping_id#24, 0) & 1) as tinyint) = 0) THEN s_state#22 END AS _w2#32] (44) CometColumnarExchange -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: hashpartitioning(_w1#31, _w2#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (45) CometSort -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32], [_w1#31 ASC NULLS FIRST, _w2#32 ASC NULLS FIRST, _w0#30 DESC NULLS LAST] (46) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] (47) Window -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] +Input [7]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32] +Arguments: [rank(_w0#30) windowspecdefinition(_w1#31, _w2#32, _w0#30 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#33], [_w1#31, _w2#32], [_w0#30 DESC NULLS LAST] (48) Project [codegen id : 7] -Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] +Output [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Input [8]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, _w0#30, _w1#31, _w2#32, rank_within_parent#33] (49) TakeOrderedAndProject -Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Input [5]: [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] +Arguments: 100, [lochierarchy#29 DESC NULLS LAST, CASE WHEN (lochierarchy#29 = 0) THEN s_state#22 END ASC NULLS FIRST, rank_within_parent#33 ASC NULLS FIRST], [total_sum#28, s_state#22, s_county#23, lochierarchy#29, rank_within_parent#33] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt index 90bcb79a34..45a2c7a669 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt @@ -37,9 +37,9 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -58,4 +58,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 53 eligible operators (64%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index f6953d47ff..80f0cfc8c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -54,10 +54,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] CometExchange [s_state] #7 CometHashAggregate [ss_net_profit] [s_state,sum] CometProject [ss_net_profit,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt index 17435b896f..00ff66d3eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/explain.txt @@ -1,42 +1,41 @@ == Physical Plan == -* CometColumnarToRow (38) -+- CometSort (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * CometColumnarToRow (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) +* CometColumnarToRow (37) ++- CometSort (36) + +- CometExchange (35) + +- CometHashAggregate (34) + +- CometExchange (33) + +- CometHashAggregate (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometBroadcastExchange (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : +- CometUnion (23) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) + : : +- CometBroadcastExchange (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : +- ReusedExchange (15) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (19) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (20) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -197,56 +196,51 @@ Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] - -(35) HashAggregate [codegen id : 1] +(34) CometHashAggregate Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#14))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#5 AS brand#42, t_hour#36, t_minute#37, MakeDecimal(sum(UnscaledValue(ext_price#14))#40,17,2) AS ext_price#43] -(36) CometColumnarExchange -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(35) CometExchange +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(37) CometSort -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] -Arguments: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43], [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST] +(36) CometSort +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] +Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] -(38) CometColumnarToRow [codegen id : 2] -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] +(37) CometColumnarToRow [codegen id : 1] +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (43) -+- * CometColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(38) 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: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter +(39) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) -(41) CometProject +(40) CometProject Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(43) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#11] 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-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt index 4f18294603..25ea15c536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/extended.txt @@ -1,54 +1,53 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, t_hour#5, t_minute#6, MakeDecimal(sum(UnscaledValue(ext_price#7))#8,17,2) AS ext_price#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- 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 - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- 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 + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 44 out of 49 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt index 45ecc07450..63e7c353ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71.native_iceberg_compat/simplified.txt @@ -1,52 +1,49 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] - CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] - CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_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 [d_date_sk] #5 - 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 [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometExchange [ext_price,brand_id] #1 + CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] + CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion [ext_price,sold_item_sk,time_sk] + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_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 [d_date_sk] #5 + 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 [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt index 17435b896f..00ff66d3eb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -1,42 +1,41 @@ == Physical Plan == -* CometColumnarToRow (38) -+- CometSort (37) - +- CometColumnarExchange (36) - +- * HashAggregate (35) - +- * CometColumnarToRow (34) - +- CometExchange (33) - +- CometHashAggregate (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometBroadcastExchange (4) - : : +- CometProject (3) - : : +- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : +- CometUnion (23) - : :- CometProject (12) - : : +- CometBroadcastHashJoin (11) - : : :- CometFilter (6) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) - : : +- CometBroadcastExchange (10) - : : +- CometProject (9) - : : +- CometFilter (8) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) - : :- CometProject (17) - : : +- CometBroadcastHashJoin (16) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) - : : +- ReusedExchange (15) - : +- CometProject (22) - : +- CometBroadcastHashJoin (21) - : :- CometFilter (19) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - : +- ReusedExchange (20) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) +* CometColumnarToRow (37) ++- CometSort (36) + +- CometExchange (35) + +- CometHashAggregate (34) + +- CometExchange (33) + +- CometHashAggregate (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometProject (25) + : +- CometBroadcastHashJoin (24) + : :- CometBroadcastExchange (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : +- CometUnion (23) + : :- CometProject (12) + : : +- CometBroadcastHashJoin (11) + : : :- CometFilter (6) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (5) + : : +- CometBroadcastExchange (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (7) + : :- CometProject (17) + : : +- CometBroadcastHashJoin (16) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (13) + : : +- ReusedExchange (15) + : +- CometProject (22) + : +- CometBroadcastHashJoin (21) + : :- CometFilter (19) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (20) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim (26) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -197,56 +196,51 @@ Functions [1]: [partial_sum(UnscaledValue(ext_price#14))] Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Arguments: hashpartitioning(i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] - -(35) HashAggregate [codegen id : 1] +(34) CometHashAggregate Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#14))#40] -Results [5]: [i_brand_id#2 AS brand_id#41, i_brand#5 AS brand#42, t_hour#36, t_minute#37, MakeDecimal(sum(UnscaledValue(ext_price#14))#40,17,2) AS ext_price#43] -(36) CometColumnarExchange -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] -Arguments: rangepartitioning(ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(35) CometExchange +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(37) CometSort -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] -Arguments: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43], [ext_price#43 DESC NULLS LAST, brand_id#41 ASC NULLS FIRST] +(36) CometSort +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] +Arguments: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42], [ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST] -(38) CometColumnarToRow [codegen id : 2] -Input [5]: [brand_id#41, brand#42, t_hour#36, t_minute#37, ext_price#43] +(37) CometColumnarToRow [codegen id : 1] +Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 5 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (43) -+- * CometColumnarToRow (42) - +- CometProject (41) - +- CometFilter (40) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (39) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(38) 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: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(40) CometFilter +(39) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((isnotnull(d_moy#13) AND isnotnull(d_year#12)) AND (d_moy#13 = 11)) AND (d_year#12 = 1999)) AND isnotnull(d_date_sk#11)) -(41) CometProject +(40) CometProject Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(42) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(43) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#11] 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-v1_4-spark4_0/q71/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt index 4f18294603..25ea15c536 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/extended.txt @@ -1,54 +1,53 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_brand_id#1 AS brand_id#2, i_brand#3 AS brand#4, t_hour#5, t_minute#6, MakeDecimal(sum(UnscaledValue(ext_price#7))#8,17,2) AS ext_price#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometUnion - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- 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 - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometUnion + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- 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 + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim -Comet accelerated 44 out of 49 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 49 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt index 45ecc07450..63e7c353ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -1,52 +1,49 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 - WholeStageCodegen (1) - HashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum] [sum(UnscaledValue(ext_price)),brand_id,brand,ext_price,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 - CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] - CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] - CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] - CometProject [i_brand_id,i_brand,ext_price,time_sk] - CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] - CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 - CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] - CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] - CometUnion [ext_price,sold_item_sk,time_sk] - CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] - CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_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 [d_date_sk] #5 - 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 [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] - CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] - CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] - CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 - CometProject [t_time_sk,t_hour,t_minute] - CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] - CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] + CometExchange [ext_price,brand_id] #1 + CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] + CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 + CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] + CometProject [i_brand_id,i_brand,ext_price,t_hour,t_minute] + CometBroadcastHashJoin [i_brand_id,i_brand,ext_price,time_sk,t_time_sk,t_hour,t_minute] + CometProject [i_brand_id,i_brand,ext_price,time_sk] + CometBroadcastHashJoin [i_item_sk,i_brand_id,i_brand,ext_price,sold_item_sk,time_sk] + CometBroadcastExchange [i_item_sk,i_brand_id,i_brand] #3 + CometProject [i_brand] [i_item_sk,i_brand_id,i_brand] + CometFilter [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + CometUnion [ext_price,sold_item_sk,time_sk] + CometProject [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,d_date_sk] + CometFilter [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_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 [d_date_sk] #5 + 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 [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,d_date_sk] + CometFilter [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometProject [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] [ext_price,sold_item_sk,time_sk] + CometBroadcastHashJoin [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,d_date_sk] + CometFilter [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [t_time_sk,t_hour,t_minute] #6 + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_time_sk,t_hour,t_minute,t_meal_time] + CometScan [native_iceberg_compat] parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt index c5086109a5..ce451747c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] 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 : 4] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] - -(17) HashAggregate [codegen id : 4] +(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))#15] -Results [2]: [c_customer_id#5 AS customer_id#16, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#15,17,2) AS year_total#17] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#16, year_total#17] -Condition : (isnotnull(year_total#17) AND (year_total#17 > 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#18, c_customer_id#19, c_first_name#20, c_last_name#21] +(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#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) -(21) CometProject -Input [4]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Arguments: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)) AS c_customer_id#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#24] +(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#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_customer_sk#25) +(22) CometFilter +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_customer_sk#24) -(24) CometBroadcastExchange -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -(25) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24] -Right output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_sk#18], [ss_customer_sk#25], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] +Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight -(26) CometProject -Input [7]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24, ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_year#30] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(28) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: [d_date_sk#29, d_year#30] - -(30) CometBroadcastHashJoin -Left output [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] -Right output [2]: [d_date_sk#29, d_year#30] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight - -(31) CometProject -Input [7]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] - -(32) CometHashAggregate -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#26))] - -(33) CometExchange -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] - -(35) HashAggregate [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [sum(UnscaledValue(ss_net_paid#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#26))#15] -Results [4]: [c_customer_id#22 AS customer_id#32, c_first_name#23 AS customer_first_name#33, c_last_name#24 AS customer_last_name#34, MakeDecimal(sum(UnscaledValue(ss_net_paid#26))#15,17,2) AS year_total#35] - -(36) BroadcastExchange -Input [4]: [customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#32] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] +(27) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28, d_year#29] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_year#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] + +(31) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] + +(32) CometExchange +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight + +(36) 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 -(39) CometFilter -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#36) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)))) +(37) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) -(40) CometProject -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Arguments: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42], [c_customer_sk#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)) AS c_customer_id#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#38, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#39, 30)) AS c_last_name#42] +(38) 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, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(39) 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#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_customer_sk#43) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) + +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(43) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] +Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42] -Right output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_sk#36], [ws_bill_customer_sk#43], Inner, BuildRight +(43) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -(45) CometProject -Input [7]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42, ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#46, d_year#47] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#47, d_year#48] +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] -Right output [2]: [d_date_sk#47, d_year#48] -Arguments: [ws_sold_date_sk#45], [d_date_sk#47], Inner, BuildRight +(46) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -(48) CometProject -Input [7]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45, d_date_sk#47, d_year#48] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] +(47) CometHashAggregate +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))] + +(48) CometExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#44))] - -(50) CometExchange -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] - -(52) HashAggregate [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [sum(UnscaledValue(ws_net_paid#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#44))#50] -Results [2]: [c_customer_id#40 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#44))#50,17,2) AS year_total#52] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [7]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52] -Input [8]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, customer_id#51, year_total#52] - -(57) 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] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +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))] + +(50) CometFilter +Input [2]: [customer_id#49, year_total#50] +Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#49, year_total#50] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Right output [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] +Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(55) CometFilter +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) -(59) 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#57, c_first_name#58, c_last_name#59], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#59] +(56) CometProject +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +(58) CometFilter +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#58) + +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] +Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight -(62) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(61) CometProject +Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#62, d_year#63] (63) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59] -Right output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#60], Inner, BuildRight +Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] +Right output [2]: [d_date_sk#62, d_year#63] +Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight (64) CometProject -Input [7]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59, ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#64, d_year#65] - -(66) CometBroadcastHashJoin -Left output [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] -Right output [2]: [d_date_sk#64, d_year#65] -Arguments: [ws_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(67) CometProject -Input [7]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] - -(68) CometHashAggregate -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] - -(69) CometExchange -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] - -(71) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#50] -Results [2]: [c_customer_id#57 AS customer_id#67, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#50,17,2) AS year_total#68] - -(72) BroadcastExchange -Input [2]: [customer_id#67, year_total#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#67] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#68 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END) - -(74) Project [codegen id : 4] -Output [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52, customer_id#67, year_total#68] - -(75) TakeOrderedAndProject -Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Arguments: 100, [customer_id#32 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] +Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] + +(65) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] + +(66) CometExchange +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#65, year_total#66] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] +Right output [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(71) CometTakeOrderedAndProject +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_id#31 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_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(72) CometColumnarToRow [codegen id : 1] +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 (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#29, d_year#30] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(81) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) +(78) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#30] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_year#29] -(83) BroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt index 966f9a40f3..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#3))#4,17,2) AS year_total#5)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#6 AS customer_id#7, c_first_name#8 AS customer_first_name#9, c_last_name#10 AS customer_last_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#12))#4,17,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue(ws_net_paid#16))#17,17,2) AS year_total#18)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#19 AS customer_id#20, MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#17,17,2) AS year_total#22)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt index 58b40d0c50..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74.native_iceberg_compat/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-v1_4-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt index c5086109a5..ce451747c0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] 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 : 4] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] - -(17) HashAggregate [codegen id : 4] +(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))#15] -Results [2]: [c_customer_id#5 AS customer_id#16, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#15,17,2) AS year_total#17] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#16, year_total#17] -Condition : (isnotnull(year_total#17) AND (year_total#17 > 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#18, c_customer_id#19, c_first_name#20, c_last_name#21] +(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#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) -(21) CometProject -Input [4]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Arguments: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)) AS c_customer_id#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#24] +(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#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_customer_sk#25) +(22) CometFilter +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_customer_sk#24) -(24) CometBroadcastExchange -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -(25) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24] -Right output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_sk#18], [ss_customer_sk#25], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] +Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight -(26) CometProject -Input [7]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24, ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_year#30] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(28) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: [d_date_sk#29, d_year#30] - -(30) CometBroadcastHashJoin -Left output [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] -Right output [2]: [d_date_sk#29, d_year#30] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight - -(31) CometProject -Input [7]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] - -(32) CometHashAggregate -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#26))] - -(33) CometExchange -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] - -(35) HashAggregate [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [sum(UnscaledValue(ss_net_paid#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#26))#15] -Results [4]: [c_customer_id#22 AS customer_id#32, c_first_name#23 AS customer_first_name#33, c_last_name#24 AS customer_last_name#34, MakeDecimal(sum(UnscaledValue(ss_net_paid#26))#15,17,2) AS year_total#35] - -(36) BroadcastExchange -Input [4]: [customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#32] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] +(27) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28, d_year#29] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_year#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] + +(31) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] + +(32) CometExchange +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight + +(36) 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 -(39) CometFilter -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#36) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)))) +(37) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) -(40) CometProject -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Arguments: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42], [c_customer_sk#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)) AS c_customer_id#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#38, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#39, 30)) AS c_last_name#42] +(38) 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, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(39) 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#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_customer_sk#43) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) + +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(43) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] +Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42] -Right output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_sk#36], [ws_bill_customer_sk#43], Inner, BuildRight +(43) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -(45) CometProject -Input [7]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42, ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#46, d_year#47] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#47, d_year#48] +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] -Right output [2]: [d_date_sk#47, d_year#48] -Arguments: [ws_sold_date_sk#45], [d_date_sk#47], Inner, BuildRight +(46) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -(48) CometProject -Input [7]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45, d_date_sk#47, d_year#48] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] +(47) CometHashAggregate +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))] + +(48) CometExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#44))] - -(50) CometExchange -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] - -(52) HashAggregate [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [sum(UnscaledValue(ws_net_paid#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#44))#50] -Results [2]: [c_customer_id#40 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#44))#50,17,2) AS year_total#52] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [7]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52] -Input [8]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, customer_id#51, year_total#52] - -(57) 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] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +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))] + +(50) CometFilter +Input [2]: [customer_id#49, year_total#50] +Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#49, year_total#50] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Right output [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] +Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(55) CometFilter +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) -(59) 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#57, c_first_name#58, c_last_name#59], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#59] +(56) CometProject +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +(58) CometFilter +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#58) + +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] +Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight -(62) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(61) CometProject +Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#62, d_year#63] (63) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59] -Right output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#60], Inner, BuildRight +Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] +Right output [2]: [d_date_sk#62, d_year#63] +Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight (64) CometProject -Input [7]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59, ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#64, d_year#65] - -(66) CometBroadcastHashJoin -Left output [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] -Right output [2]: [d_date_sk#64, d_year#65] -Arguments: [ws_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(67) CometProject -Input [7]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] - -(68) CometHashAggregate -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] - -(69) CometExchange -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] - -(71) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#50] -Results [2]: [c_customer_id#57 AS customer_id#67, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#50,17,2) AS year_total#68] - -(72) BroadcastExchange -Input [2]: [customer_id#67, year_total#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#67] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#68 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END) - -(74) Project [codegen id : 4] -Output [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52, customer_id#67, year_total#68] - -(75) TakeOrderedAndProject -Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Arguments: 100, [customer_id#32 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] +Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] + +(65) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] + +(66) CometExchange +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#65, year_total#66] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] +Right output [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(71) CometTakeOrderedAndProject +Input [3]: [customer_id#31, customer_first_name#32, customer_last_name#33] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#31 ASC NULLS FIRST,customer_id#31 ASC NULLS FIRST,customer_id#31 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_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST, customer_id#31 ASC NULLS FIRST], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(72) CometColumnarToRow [codegen id : 1] +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 (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#29, d_year#30] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(81) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) +(78) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#30] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_year#29] -(83) BroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt index 966f9a40f3..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#3))#4,17,2) AS year_total#5)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#6 AS customer_id#7, c_first_name#8 AS customer_first_name#9, c_last_name#10 AS customer_last_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#12))#4,17,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue(ws_net_paid#16))#17,17,2) AS year_total#18)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#19 AS customer_id#20, MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#17,17,2) AS year_total#22)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt index 58b40d0c50..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt index 9ccef017a7..02ac3ecdc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/explain.txt @@ -1,130 +1,128 @@ == Physical Plan == -* CometColumnarToRow (126) -+- CometTakeOrderedAndProject (125) - +- CometProject (124) - +- CometSortMergeJoin (123) - :- CometSort (67) - : +- CometColumnarExchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * CometColumnarToRow (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- 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) - : : +- 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 (122) - +- CometColumnarExchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (78) - : : +- CometProject (77) - : : +- CometBroadcastHashJoin (76) - : : :- CometProject (72) - : : : +- CometBroadcastHashJoin (71) - : : : :- CometFilter (69) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (70) - : : +- CometBroadcastExchange (75) - : : +- CometFilter (74) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (97) - : +- CometSortMergeJoin (96) - : :- CometSort (93) - : : +- CometExchange (92) - : : +- CometProject (91) - : : +- CometBroadcastHashJoin (90) - : : :- CometProject (88) - : : : +- CometBroadcastHashJoin (87) - : : : :- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (86) - : : +- ReusedExchange (89) - : +- CometSort (95) - : +- ReusedExchange (94) - +- CometProject (111) - +- CometSortMergeJoin (110) - :- CometSort (107) - : +- CometExchange (106) - : +- CometProject (105) - : +- CometBroadcastHashJoin (104) - : :- CometProject (102) - : : +- CometBroadcastHashJoin (101) - : : :- CometFilter (99) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (98) - : : +- ReusedExchange (100) - : +- ReusedExchange (103) - +- CometSort (109) - +- ReusedExchange (108) +* 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) + : : : +- 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 (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 @@ -411,310 +409,300 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) CometColumnarToRow [codegen id : 1] -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] - -(64) HashAggregate [codegen id : 1] +(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))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#64 AS sales_cnt#66, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#67] -(65) Filter [codegen id : 1] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Condition : isnotnull(sales_cnt#66) +(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) -(66) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(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=9] -(67) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67], [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] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] +(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 [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#68) +(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) -(70) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(71) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Right output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68], [i_item_sk#74], Inner, BuildRight +(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 -(72) CometProject -Input [10]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(74) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(75) CometBroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] -Arguments: [d_date_sk#79, d_year#80] +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(76) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Right output [2]: [d_date_sk#79, d_year#80] -Arguments: [cs_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight +(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) CometProject -Input [11]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_date_sk#79, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] +(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) CometExchange -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: hashpartitioning(cs_order_number#69, cs_item_sk#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(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, CometNativeShuffle, [plan_id=10] -(79) CometSort -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_order_number#69 ASC NULLS FIRST, cs_item_sk#68 ASC NULLS FIRST] +(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] -Output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] +(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 -Input [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84], [cr_order_number#82 ASC NULLS FIRST, cr_item_sk#81 ASC NULLS FIRST] +(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 -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Right output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cs_order_number#69, cs_item_sk#68], [cr_order_number#82, cr_item_sk#81], LeftOuter +(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 -Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86], [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] +(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#83, sales_amt#84], [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#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(84) CometFilter +Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Condition : isnotnull(ss_item_sk#85) -(86) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(85) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(87) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight -(88) CometProject -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(87) CometProject +Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(89) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#98, d_year#99] +(88) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#96, d_year#97] -(90) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Right output [2]: [d_date_sk#98, d_year#99] -Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Right output [2]: [d_date_sk#96, d_year#97] +Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight -(91) CometProject -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(90) CometProject +Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -(92) CometExchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(91) CometExchange +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(93) CometSort -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST] +(92) CometSort +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] -(94) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(93) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -(95) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +(94) CometSort +Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] -(96) CometSortMergeJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Right output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [ss_ticket_number#88, ss_item_sk#87], [sr_ticket_number#101, sr_item_sk#100], LeftOuter +(95) CometSortMergeJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter -(97) CometProject -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105], [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#104, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#105] +(96) CometProject +Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#110), dynamicpruningexpression(ws_sold_date_sk#110 IN dynamicpruning#111)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter -Input [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Condition : isnotnull(ws_item_sk#106) +(98) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(100) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(99) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(101) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Right output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106], [i_item_sk#112], Inner, BuildRight +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(102) CometProject -Input [10]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(101) CometProject +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(103) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#117, d_year#118] +(102) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#115, d_year#116] -(104) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Right output [2]: [d_date_sk#117, d_year#118] -Arguments: [ws_sold_date_sk#110], [d_date_sk#117], Inner, BuildRight +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(105) CometProject -Input [11]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_date_sk#117, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] +(104) CometProject +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(106) CometExchange -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: hashpartitioning(ws_order_number#107, ws_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(105) CometExchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(107) CometSort -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_order_number#107 ASC NULLS FIRST, ws_item_sk#106 ASC NULLS FIRST] +(106) CometSort +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] -(108) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] +(107) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(109) CometSort -Input [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122], [wr_order_number#120 ASC NULLS FIRST, wr_item_sk#119 ASC NULLS FIRST] +(108) CometSort +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] -(110) CometSortMergeJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Right output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [ws_order_number#107, ws_item_sk#106], [wr_order_number#120, wr_item_sk#119], LeftOuter +(109) CometSortMergeJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter -(111) CometProject -Input [13]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118, wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124], [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, (ws_quantity#108 - coalesce(wr_return_quantity#121, 0)) AS sales_cnt#123, (ws_ext_sales_price#109 - coalesce(wr_return_amt#122, 0.00)) AS sales_amt#124] +(110) CometProject +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] -(112) CometUnion -Child 0 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Child 1 Input [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105] -Child 2 Input [7]: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124] +(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#83, sales_amt#84] +Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] +Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] -(113) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(114) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(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#83, sales_amt#84] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(116) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] - -(117) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +(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#83, sales_amt#84] +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#83), partial_sum(UnscaledValue(sales_amt#84))] -(118) CometColumnarToRow [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] +(116) CometExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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] -(119) HashAggregate [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] -Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] +(117) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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#83), sum(UnscaledValue(sales_amt#84))] -(120) Filter [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Condition : isnotnull(sales_cnt#127) +(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#125, sales_amt#126] +Condition : isnotnull(sales_cnt#125) -(121) CometColumnarExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(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#125, sales_amt#126] +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] -(122) CometSort -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128], [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST] +(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#125, sales_amt#126] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [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] -(123) 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#66, sales_amt#67] -Right output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], Inner, ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) +(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#125, sales_amt#126] +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#125 as decimal(17,2))) < 0.90000000000000000000) -(124) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +(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#125, sales_amt#126] +Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] -(125) CometTakeOrderedAndProject -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#133 ASC NULLS FIRST], output=[prev_year#129,year#130,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#131,curr_yr_cnt#132,sales_cnt_diff#133,sales_amt_diff#134]), [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], 100, 0, [sales_cnt_diff#133 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(123) CometTakeOrderedAndProject +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -(126) CometColumnarToRow [codegen id : 3] -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(124) CometColumnarToRow [codegen id : 1] +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (130) -+- * CometColumnarToRow (129) - +- CometFilter (128) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (127) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(127) 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 -(128) 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)) -(129) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(130) 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=16] @@ -722,33 +710,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometFilter (132) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (131) +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) -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(132) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(133) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#80] +(131) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(134) BroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] +(132) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] 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#91 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#110 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt index 42b42735de..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/extended.txt @@ -3,172 +3,170 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2, i_class_id#3, i_category_id#4, i_manufact_id#5, sum(sales_cnt#6)#7 AS sales_cnt#8, MakeDecimal(sum(UnscaledValue(sales_amt#9))#10,18,2) AS sales_amt#11)] - : +- CometColumnarToRow - : +- CometExchange + : +- CometExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- 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 + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +- CometSort - +- CometColumnarExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#12, i_brand_id#13, i_class_id#14, i_category_id#15, i_manufact_id#16, sum(sales_cnt#17)#7 AS sales_cnt#18, MakeDecimal(sum(UnscaledValue(sales_amt#19))#10,18,2) AS sales_amt#20)] - +- CometColumnarToRow - +- CometExchange + +- CometExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- 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 - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- 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 + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometProject + +- CometSortMergeJoin + :- CometSort + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 155 out of 167 eligible operators (92%). Final plan contains 5 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-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt index 9a3700b0b1..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75.native_iceberg_compat/simplified.txt @@ -1,150 +1,144 @@ -WholeStageCodegen (3) +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] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [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] CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (1) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - 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] #3 - 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] #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] #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] - 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] #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] - 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] #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] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + 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] #2 + 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] #3 + 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] #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] #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] + 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] #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] + 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] #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] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (2) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - 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] #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] - 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] #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] - 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] #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] - 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] #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] #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] #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] + 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] #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] + 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] #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] + 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] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt index 9ccef017a7..02ac3ecdc2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt @@ -1,130 +1,128 @@ == Physical Plan == -* CometColumnarToRow (126) -+- CometTakeOrderedAndProject (125) - +- CometProject (124) - +- CometSortMergeJoin (123) - :- CometSort (67) - : +- CometColumnarExchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * CometColumnarToRow (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- 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) - : : +- 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 (122) - +- CometColumnarExchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (78) - : : +- CometProject (77) - : : +- CometBroadcastHashJoin (76) - : : :- CometProject (72) - : : : +- CometBroadcastHashJoin (71) - : : : :- CometFilter (69) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (70) - : : +- CometBroadcastExchange (75) - : : +- CometFilter (74) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (97) - : +- CometSortMergeJoin (96) - : :- CometSort (93) - : : +- CometExchange (92) - : : +- CometProject (91) - : : +- CometBroadcastHashJoin (90) - : : :- CometProject (88) - : : : +- CometBroadcastHashJoin (87) - : : : :- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (86) - : : +- ReusedExchange (89) - : +- CometSort (95) - : +- ReusedExchange (94) - +- CometProject (111) - +- CometSortMergeJoin (110) - :- CometSort (107) - : +- CometExchange (106) - : +- CometProject (105) - : +- CometBroadcastHashJoin (104) - : :- CometProject (102) - : : +- CometBroadcastHashJoin (101) - : : :- CometFilter (99) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (98) - : : +- ReusedExchange (100) - : +- ReusedExchange (103) - +- CometSort (109) - +- ReusedExchange (108) +* 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) + : : : +- 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 (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 @@ -411,310 +409,300 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) CometColumnarToRow [codegen id : 1] -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] - -(64) HashAggregate [codegen id : 1] +(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))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#64 AS sales_cnt#66, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#67] -(65) Filter [codegen id : 1] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Condition : isnotnull(sales_cnt#66) +(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) -(66) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(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=9] -(67) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67], [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] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] +(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 [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#68) +(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) -(70) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(71) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Right output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68], [i_item_sk#74], Inner, BuildRight +(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 -(72) CometProject -Input [10]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(74) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(75) CometBroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] -Arguments: [d_date_sk#79, d_year#80] +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(76) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Right output [2]: [d_date_sk#79, d_year#80] -Arguments: [cs_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight +(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) CometProject -Input [11]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_date_sk#79, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] +(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) CometExchange -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: hashpartitioning(cs_order_number#69, cs_item_sk#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(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, CometNativeShuffle, [plan_id=10] -(79) CometSort -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_order_number#69 ASC NULLS FIRST, cs_item_sk#68 ASC NULLS FIRST] +(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] -Output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] +(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 -Input [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84], [cr_order_number#82 ASC NULLS FIRST, cr_item_sk#81 ASC NULLS FIRST] +(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 -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Right output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cs_order_number#69, cs_item_sk#68], [cr_order_number#82, cr_item_sk#81], LeftOuter +(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 -Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86], [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] +(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#83, sales_amt#84], [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#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(84) CometFilter +Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Condition : isnotnull(ss_item_sk#85) -(86) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(85) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(87) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight -(88) CometProject -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(87) CometProject +Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(89) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#98, d_year#99] +(88) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#96, d_year#97] -(90) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Right output [2]: [d_date_sk#98, d_year#99] -Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Right output [2]: [d_date_sk#96, d_year#97] +Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight -(91) CometProject -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(90) CometProject +Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -(92) CometExchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(91) CometExchange +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(93) CometSort -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST] +(92) CometSort +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] -(94) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(93) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -(95) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +(94) CometSort +Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] -(96) CometSortMergeJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Right output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [ss_ticket_number#88, ss_item_sk#87], [sr_ticket_number#101, sr_item_sk#100], LeftOuter +(95) CometSortMergeJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter -(97) CometProject -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105], [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#104, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#105] +(96) CometProject +Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#110), dynamicpruningexpression(ws_sold_date_sk#110 IN dynamicpruning#111)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter -Input [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Condition : isnotnull(ws_item_sk#106) +(98) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(100) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(99) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(101) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Right output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106], [i_item_sk#112], Inner, BuildRight +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(102) CometProject -Input [10]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(101) CometProject +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(103) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#117, d_year#118] +(102) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#115, d_year#116] -(104) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Right output [2]: [d_date_sk#117, d_year#118] -Arguments: [ws_sold_date_sk#110], [d_date_sk#117], Inner, BuildRight +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(105) CometProject -Input [11]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_date_sk#117, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] +(104) CometProject +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(106) CometExchange -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: hashpartitioning(ws_order_number#107, ws_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(105) CometExchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(107) CometSort -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_order_number#107 ASC NULLS FIRST, ws_item_sk#106 ASC NULLS FIRST] +(106) CometSort +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] -(108) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] +(107) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(109) CometSort -Input [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122], [wr_order_number#120 ASC NULLS FIRST, wr_item_sk#119 ASC NULLS FIRST] +(108) CometSort +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] -(110) CometSortMergeJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Right output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [ws_order_number#107, ws_item_sk#106], [wr_order_number#120, wr_item_sk#119], LeftOuter +(109) CometSortMergeJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter -(111) CometProject -Input [13]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118, wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124], [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, (ws_quantity#108 - coalesce(wr_return_quantity#121, 0)) AS sales_cnt#123, (ws_ext_sales_price#109 - coalesce(wr_return_amt#122, 0.00)) AS sales_amt#124] +(110) CometProject +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] -(112) CometUnion -Child 0 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Child 1 Input [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105] -Child 2 Input [7]: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124] +(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#83, sales_amt#84] +Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] +Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] -(113) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(114) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(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#83, sales_amt#84] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(116) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] - -(117) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +(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#83, sales_amt#84] +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#83), partial_sum(UnscaledValue(sales_amt#84))] -(118) CometColumnarToRow [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] +(116) CometExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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] -(119) HashAggregate [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] -Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] +(117) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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#83), sum(UnscaledValue(sales_amt#84))] -(120) Filter [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Condition : isnotnull(sales_cnt#127) +(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#125, sales_amt#126] +Condition : isnotnull(sales_cnt#125) -(121) CometColumnarExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(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#125, sales_amt#126] +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] -(122) CometSort -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128], [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST] +(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#125, sales_amt#126] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [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] -(123) 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#66, sales_amt#67] -Right output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], Inner, ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) +(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#125, sales_amt#126] +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#125 as decimal(17,2))) < 0.90000000000000000000) -(124) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +(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#125, sales_amt#126] +Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] -(125) CometTakeOrderedAndProject -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#133 ASC NULLS FIRST], output=[prev_year#129,year#130,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#131,curr_yr_cnt#132,sales_cnt_diff#133,sales_amt_diff#134]), [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], 100, 0, [sales_cnt_diff#133 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(123) CometTakeOrderedAndProject +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -(126) CometColumnarToRow [codegen id : 3] -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(124) CometColumnarToRow [codegen id : 1] +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (130) -+- * CometColumnarToRow (129) - +- CometFilter (128) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (127) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(127) 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 -(128) 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)) -(129) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(130) 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=16] @@ -722,33 +710,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometFilter (132) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (131) +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) -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(132) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(133) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#80] +(131) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(134) BroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] +(132) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] 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#91 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#110 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt index 42b42735de..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/extended.txt @@ -3,172 +3,170 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2, i_class_id#3, i_category_id#4, i_manufact_id#5, sum(sales_cnt#6)#7 AS sales_cnt#8, MakeDecimal(sum(UnscaledValue(sales_amt#9))#10,18,2) AS sales_amt#11)] - : +- CometColumnarToRow - : +- CometExchange + : +- CometExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- 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 + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +- CometSort - +- CometColumnarExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#12, i_brand_id#13, i_class_id#14, i_category_id#15, i_manufact_id#16, sum(sales_cnt#17)#7 AS sales_cnt#18, MakeDecimal(sum(UnscaledValue(sales_amt#19))#10,18,2) AS sales_amt#20)] - +- CometColumnarToRow - +- CometExchange + +- CometExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- 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 - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- 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 + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometProject + +- CometSortMergeJoin + :- CometSort + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 155 out of 167 eligible operators (92%). Final plan contains 5 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-v1_4-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt index 9a3700b0b1..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -1,150 +1,144 @@ -WholeStageCodegen (3) +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] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [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] CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (1) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - 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] #3 - 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] #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] #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] - 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] #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] - 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] #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] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + 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] #2 + 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] #3 + 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] #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] #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] + 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] #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] + 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] #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] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (2) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - 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] #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] - 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] #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] - 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] #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] - 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] #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] #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] #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] + 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] #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] + 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] #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] + 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] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt index 6de2cd18d8..8b346eb5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (35) -+- * HashAggregate (34) - +- * CometColumnarToRow (33) +* CometColumnarToRow (35) ++- CometTakeOrderedAndProject (34) + +- CometHashAggregate (33) +- CometExchange (32) +- CometHashAggregate (31) +- CometUnion (30) @@ -186,17 +186,15 @@ Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13)) Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(33) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] - -(34) HashAggregate [codegen id : 1] +(33) CometHashAggregate Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] -Aggregate Attributes [2]: [count(1)#40, sum(UnscaledValue(ext_sales_price#13))#41] -Results [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count(1)#40 AS sales_cnt#42, MakeDecimal(sum(UnscaledValue(ext_sales_price#13))#41,17,2) AS sales_amt#43] -(35) TakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#42, sales_amt#43] -Arguments: 100, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#42, sales_amt#43] +(34) CometTakeOrderedAndProject +Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] + +(35) CometColumnarToRow [codegen id : 1] +Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt index 1abe39e093..b56a6590db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(channel#1, col_name#2, d_year#3, d_qoy#4, i_category#5, count(1)#6 AS sales_cnt#7, MakeDecimal(sum(UnscaledValue(ext_sales_price#8))#9,17,2) AS sales_amt#10)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometUnion @@ -44,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 44 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt index 5182643eac..b5449831cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (1) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt index 6de2cd18d8..8b346eb5c9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (35) -+- * HashAggregate (34) - +- * CometColumnarToRow (33) +* CometColumnarToRow (35) ++- CometTakeOrderedAndProject (34) + +- CometHashAggregate (33) +- CometExchange (32) +- CometHashAggregate (31) +- CometUnion (30) @@ -186,17 +186,15 @@ Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#13)) Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] Arguments: hashpartitioning(channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(33) CometColumnarToRow [codegen id : 1] -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] - -(34) HashAggregate [codegen id : 1] +(33) CometHashAggregate Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count#38, sum#39] Keys [5]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7] Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#13))] -Aggregate Attributes [2]: [count(1)#40, sum(UnscaledValue(ext_sales_price#13))#41] -Results [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, count(1)#40 AS sales_cnt#42, MakeDecimal(sum(UnscaledValue(ext_sales_price#13))#41,17,2) AS sales_amt#43] -(35) TakeOrderedAndProject -Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#42, sales_amt#43] -Arguments: 100, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#42, sales_amt#43] +(34) CometTakeOrderedAndProject +Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#11 ASC NULLS FIRST,col_name#12 ASC NULLS FIRST,d_year#9 ASC NULLS FIRST,d_qoy#10 ASC NULLS FIRST,i_category#7 ASC NULLS FIRST], output=[channel#11,col_name#12,d_year#9,d_qoy#10,i_category#7,sales_cnt#40,sales_amt#41]), [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41], 100, 0, [channel#11 ASC NULLS FIRST, col_name#12 ASC NULLS FIRST, d_year#9 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, i_category#7 ASC NULLS FIRST], [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] + +(35) CometColumnarToRow [codegen id : 1] +Input [7]: [channel#11, col_name#12, d_year#9, d_qoy#10, i_category#7, sales_cnt#40, sales_amt#41] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt index 1abe39e093..b56a6590db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(channel#1, col_name#2, d_year#3, d_qoy#4, i_category#5, count(1)#6 AS sales_cnt#7, MakeDecimal(sum(UnscaledValue(ext_sales_price#8))#9,17,2) AS sales_amt#10)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometUnion @@ -44,4 +44,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 44 eligible operators (95%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 44 eligible operators (100%). Final plan contains 1 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt index 5182643eac..b5449831cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] - WholeStageCodegen (1) - HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + CometHashAggregate [count,sum] [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt,count(1),sum(UnscaledValue(ext_sales_price))] CometExchange [channel,col_name,d_year,d_qoy,i_category] #1 CometHashAggregate [ext_sales_price] [channel,col_name,d_year,d_qoy,i_category,count,sum] CometUnion [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt index 0694929c87..bb7ed0a50c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/explain.txt @@ -1,91 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * Expand (82) - +- Union (81) - :- * Project (32) - : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : :- * HashAggregate (17) - : : +- * CometColumnarToRow (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) - : +- BroadcastExchange (30) - : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - :- * Project (51) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) - : :- BroadcastExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : +- ReusedExchange (34) - : +- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (42) - : +- ReusedExchange (43) - +- * Project (80) - +- * BroadcastHashJoin LeftOuter BuildRight (79) - :- * HashAggregate (65) - : +- * CometColumnarToRow (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : +- ReusedExchange (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (57) - +- BroadcastExchange (78) - +- * HashAggregate (77) - +- * CometColumnarToRow (76) - +- CometExchange (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (70) - : +- CometBroadcastHashJoin (69) - : :- CometFilter (67) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (66) - : +- ReusedExchange (68) - +- ReusedExchange (71) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * Expand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -161,380 +159,360 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Input [3]: [s_store_sk#8, sum#9, sum#10] Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#8, sum#9, sum#10] - -(17) HashAggregate [codegen id : 2] +(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))#11, sum(UnscaledValue(ss_net_profit#3))#12] -Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#15) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(21) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(22) CometProject -Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] -Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(24) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#21] -Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight - -(25) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] - -(26) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] - -(27) CometExchange -Input [3]: [s_store_sk#21, sum#22, sum#23] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#26, profit_loss#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None +(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] + +(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) 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) 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] + +(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 : 2] -Output [5]: [sales#13, coalesce(returns#26, 0.00) AS returns#28, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#29, store channel AS channel#30, s_store_sk#8 AS id#31] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +(31) CometColumnarToRow [codegen id : 1] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#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#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] -Right output [1]: [d_date_sk#37] -Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight +(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) CometProject -Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] -Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +(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] -(37) CometHashAggregate -Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] +(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))] -(38) CometExchange -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(39) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(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))] -(40) HashAggregate [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] -Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(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#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(44) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] -Right output [1]: [d_date_sk#48] -Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight +(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) CometProject -Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] -Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] +(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) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] - -(47) CometExchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) CometColumnarToRow -Input [2]: [sum#49, sum#50] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 4] +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(51) Project [codegen id : 4] -Output [5]: [sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#55, catalog channel AS channel#56, cs_call_center_sk#32 AS id#57] -Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] +(50) Project [codegen id : 3] +Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_web_page_sk#58) +(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) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(55) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight +(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 -(56) CometProject -Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] -Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +(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] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(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 -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(60) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -Right output [1]: [wp_web_page_sk#64] -Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(61) CometProject -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +(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 -(62) CometHashAggregate -Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] +(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] -(63) CometExchange -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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))] -(64) CometColumnarToRow [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(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) HashAggregate [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] -Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] +(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))] -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +(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#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Condition : isnotnull(wr_web_page_sk#71) +(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] -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(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 -(69) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Right output [1]: [d_date_sk#76] -Arguments: [wr_returned_date_sk#74], [d_date_sk#76], 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] -(70) CometProject -Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] -Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(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 -(72) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] -Right output [1]: [wp_web_page_sk#77] -Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], 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] -(73) CometProject -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +(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]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] - -(75) CometExchange -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] - -(77) HashAggregate [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] - -(78) BroadcastExchange -Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +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))] -(79) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wp_web_page_sk#64] -Right keys [1]: [wp_web_page_sk#77] -Join type: LeftOuter -Join condition: None +(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] -(80) Project [codegen id : 6] -Output [5]: [sales#69, coalesce(returns#82, 0.00) AS returns#84, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#85, web channel AS channel#86, wp_web_page_sk#64 AS id#87] -Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] +(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 -(81) Union +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(82) Expand [codegen id : 7] -Input [5]: [sales#13, returns#28, profit#29, channel#30, id#31] -Arguments: [[sales#13, returns#28, profit#29, channel#30, id#31, 0], [sales#13, returns#28, profit#29, channel#30, null, 1], [sales#13, returns#28, profit#29, null, null, 3]], [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] +(78) CometColumnarToRow [codegen id : 4] +Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] -(83) HashAggregate [codegen id : 7] -Input [6]: [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] -Keys [3]: [channel#88, id#89, spark_grouping_id#90] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#28), partial_sum(profit#29)] -Aggregate Attributes [6]: [sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Results [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +(79) Union -(84) CometColumnarExchange -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(channel#88, id#89, spark_grouping_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(80) Expand [codegen id : 5] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] +Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -(85) CometColumnarToRow [codegen id : 8] -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +(81) HashAggregate [codegen id : 5] +Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] +Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] +Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(86) HashAggregate [codegen id : 8] -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [3]: [channel#88, id#89, spark_grouping_id#90] -Functions [3]: [sum(sales#13), sum(returns#28), sum(profit#29)] -Aggregate Attributes [3]: [sum(sales#13)#103, sum(returns#28)#104, sum(profit#29)#105] -Results [5]: [channel#88, id#89, sum(sales#13)#103 AS sales#106, sum(returns#28)#104 AS returns#107, sum(profit#29)#105 AS profit#108] +(82) CometColumnarExchange +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(87) TakeOrderedAndProject -Input [5]: [channel#88, id#89, sales#106, returns#107, profit#108] -Arguments: 100, [channel#88 ASC NULLS FIRST, id#89 ASC NULLS FIRST], [channel#88, id#89, sales#106, returns#107, profit#108] +(83) CometColumnarToRow [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] + +(84) HashAggregate [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] +Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] +Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] + +(85) TakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] +Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (92) -+- * CometColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (88) +BroadcastExchange (90) ++- * CometColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(86) 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,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(89) CometFilter +(87) CometFilter Input [2]: [d_date_sk#6, d_date#7] Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(90) CometProject +(88) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(91) CometColumnarToRow [codegen id : 1] +(89) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(92) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#6] -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=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 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#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 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-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt index e15d35c438..4ce8365320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/extended.txt @@ -5,34 +5,33 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : +- CometColumnarToRow + :- 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 @@ -52,8 +51,8 @@ TakeOrderedAndProject :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -64,8 +63,8 @@ TakeOrderedAndProject : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -76,29 +75,28 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - +- CometColumnarToRow + +- 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 @@ -116,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 82 out of 109 eligible operators (75%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt index 10c3b0f213..a1243769e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77.native_iceberg_compat/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (8) + WholeStageCodegen (6) HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (2) - Project [sales,returns,profit,profit_loss,s_store_sk] - 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 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] + 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] #2 CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -38,32 +38,28 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_sk] #7 - 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] #4 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (4) + CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometExchange [s_store_sk] #7 + 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] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (3) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + 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] #9 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] @@ -71,9 +67,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] @@ -81,12 +77,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] #4 - WholeStageCodegen (6) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - 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 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] + 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] #11 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] @@ -100,20 +96,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [wp_web_page_sk] #14 - 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] #4 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 + 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] #14 + 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] #4 + ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt index 0694929c87..bb7ed0a50c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt @@ -1,91 +1,89 @@ == Physical Plan == -TakeOrderedAndProject (87) -+- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * Expand (82) - +- Union (81) - :- * Project (32) - : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : :- * HashAggregate (17) - : : +- * CometColumnarToRow (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) - : +- BroadcastExchange (30) - : +- * HashAggregate (29) - : +- * CometColumnarToRow (28) - : +- CometExchange (27) - : +- CometHashAggregate (26) - : +- CometProject (25) - : +- CometBroadcastHashJoin (24) - : :- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (18) - : : +- ReusedExchange (20) - : +- ReusedExchange (23) - :- * Project (51) - : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) - : :- BroadcastExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : +- ReusedExchange (34) - : +- * HashAggregate (49) - : +- * CometColumnarToRow (48) - : +- CometExchange (47) - : +- CometHashAggregate (46) - : +- CometProject (45) - : +- CometBroadcastHashJoin (44) - : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (42) - : +- ReusedExchange (43) - +- * Project (80) - +- * BroadcastHashJoin LeftOuter BuildRight (79) - :- * HashAggregate (65) - : +- * CometColumnarToRow (64) - : +- CometExchange (63) - : +- CometHashAggregate (62) - : +- CometProject (61) - : +- CometBroadcastHashJoin (60) - : :- CometProject (56) - : : +- CometBroadcastHashJoin (55) - : : :- CometFilter (53) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : +- ReusedExchange (54) - : +- CometBroadcastExchange (59) - : +- CometFilter (58) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (57) - +- BroadcastExchange (78) - +- * HashAggregate (77) - +- * CometColumnarToRow (76) - +- CometExchange (75) - +- CometHashAggregate (74) - +- CometProject (73) - +- CometBroadcastHashJoin (72) - :- CometProject (70) - : +- CometBroadcastHashJoin (69) - : :- CometFilter (67) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (66) - : +- ReusedExchange (68) - +- ReusedExchange (71) +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- * CometColumnarToRow (83) + +- CometColumnarExchange (82) + +- * HashAggregate (81) + +- * Expand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -161,380 +159,360 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Input [3]: [s_store_sk#8, sum#9, sum#10] Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#8, sum#9, sum#10] - -(17) HashAggregate [codegen id : 2] +(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))#11, sum(UnscaledValue(ss_net_profit#3))#12] -Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#15) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(21) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(22) CometProject -Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] -Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(24) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#21] -Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight - -(25) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] - -(26) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] - -(27) CometExchange -Input [3]: [s_store_sk#21, sum#22, sum#23] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#26, profit_loss#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None +(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] + +(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) 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) 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] + +(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 : 2] -Output [5]: [sales#13, coalesce(returns#26, 0.00) AS returns#28, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#29, store channel AS channel#30, s_store_sk#8 AS id#31] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] +(30) CometProject +Input [6]: [s_store_sk#8, sales#22, profit#23, s_store_sk#17, returns#20, profit_loss#21] +Arguments: [sales#22, returns#24, profit#25, channel#26, id#27], [sales#22, coalesce(returns#20, 0.00) AS returns#24, (profit#23 - coalesce(profit_loss#21, 0.00)) AS profit#25, store channel AS channel#26, s_store_sk#8 AS id#27] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +(31) CometColumnarToRow [codegen id : 1] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#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#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] -Right output [1]: [d_date_sk#37] -Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight +(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) CometProject -Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] -Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +(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] -(37) CometHashAggregate -Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] +(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))] -(38) CometExchange -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(39) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(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))] -(40) HashAggregate [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] -Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(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#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(44) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] -Right output [1]: [d_date_sk#48] -Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight +(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) CometProject -Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] -Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] +(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) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] - -(47) CometExchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(48) CometColumnarToRow -Input [2]: [sum#49, sum#50] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] -(50) BroadcastNestedLoopJoin [codegen id : 4] +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] + +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(51) Project [codegen id : 4] -Output [5]: [sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#55, catalog channel AS channel#56, cs_call_center_sk#32 AS id#57] -Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] +(50) Project [codegen id : 3] +Output [5]: [sales#36, returns#45, (profit#37 - profit_loss#46) AS profit#47, catalog channel AS channel#48, cs_call_center_sk#28 AS id#49] +Input [5]: [cs_call_center_sk#28, sales#36, profit#37, returns#45, profit_loss#46] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_web_page_sk#58) +(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) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(55) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight +(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 -(56) CometProject -Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] -Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +(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] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(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 -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(60) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -Right output [1]: [wp_web_page_sk#64] -Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(61) CometProject -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +(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 -(62) CometHashAggregate -Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] +(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] -(63) CometExchange -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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))] -(64) CometColumnarToRow [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(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) HashAggregate [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] -Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] +(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))] -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +(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#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Condition : isnotnull(wr_web_page_sk#71) +(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] -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(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 -(69) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Right output [1]: [d_date_sk#76] -Arguments: [wr_returned_date_sk#74], [d_date_sk#76], 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] -(70) CometProject -Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] -Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(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 -(72) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] -Right output [1]: [wp_web_page_sk#77] -Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], 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] -(73) CometProject -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +(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]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] - -(75) CometExchange -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] - -(77) HashAggregate [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] - -(78) BroadcastExchange -Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +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))] -(79) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wp_web_page_sk#64] -Right keys [1]: [wp_web_page_sk#77] -Join type: LeftOuter -Join condition: None +(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] -(80) Project [codegen id : 6] -Output [5]: [sales#69, coalesce(returns#82, 0.00) AS returns#84, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#85, web channel AS channel#86, wp_web_page_sk#64 AS id#87] -Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] +(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 -(81) Union +(77) CometProject +Input [6]: [wp_web_page_sk#56, sales#70, profit#71, wp_web_page_sk#65, returns#68, profit_loss#69] +Arguments: [sales#70, returns#72, profit#73, channel#74, id#75], [sales#70, coalesce(returns#68, 0.00) AS returns#72, (profit#71 - coalesce(profit_loss#69, 0.00)) AS profit#73, web channel AS channel#74, wp_web_page_sk#56 AS id#75] -(82) Expand [codegen id : 7] -Input [5]: [sales#13, returns#28, profit#29, channel#30, id#31] -Arguments: [[sales#13, returns#28, profit#29, channel#30, id#31, 0], [sales#13, returns#28, profit#29, channel#30, null, 1], [sales#13, returns#28, profit#29, null, null, 3]], [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] +(78) CometColumnarToRow [codegen id : 4] +Input [5]: [sales#70, returns#72, profit#73, channel#74, id#75] -(83) HashAggregate [codegen id : 7] -Input [6]: [sales#13, returns#28, profit#29, channel#88, id#89, spark_grouping_id#90] -Keys [3]: [channel#88, id#89, spark_grouping_id#90] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#28), partial_sum(profit#29)] -Aggregate Attributes [6]: [sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] -Results [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +(79) Union -(84) CometColumnarExchange -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Arguments: hashpartitioning(channel#88, id#89, spark_grouping_id#90, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] +(80) Expand [codegen id : 5] +Input [5]: [sales#22, returns#24, profit#25, channel#26, id#27] +Arguments: [[sales#22, returns#24, profit#25, channel#26, id#27, 0], [sales#22, returns#24, profit#25, channel#26, null, 1], [sales#22, returns#24, profit#25, null, null, 3]], [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] -(85) CometColumnarToRow [codegen id : 8] -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +(81) HashAggregate [codegen id : 5] +Input [6]: [sales#22, returns#24, profit#25, channel#76, id#77, spark_grouping_id#78] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [partial_sum(sales#22), partial_sum(returns#24), partial_sum(profit#25)] +Aggregate Attributes [6]: [sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84] +Results [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] -(86) HashAggregate [codegen id : 8] -Input [9]: [channel#88, id#89, spark_grouping_id#90, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102] -Keys [3]: [channel#88, id#89, spark_grouping_id#90] -Functions [3]: [sum(sales#13), sum(returns#28), sum(profit#29)] -Aggregate Attributes [3]: [sum(sales#13)#103, sum(returns#28)#104, sum(profit#29)#105] -Results [5]: [channel#88, id#89, sum(sales#13)#103 AS sales#106, sum(returns#28)#104 AS returns#107, sum(profit#29)#105 AS profit#108] +(82) CometColumnarExchange +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Arguments: hashpartitioning(channel#76, id#77, spark_grouping_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] -(87) TakeOrderedAndProject -Input [5]: [channel#88, id#89, sales#106, returns#107, profit#108] -Arguments: 100, [channel#88 ASC NULLS FIRST, id#89 ASC NULLS FIRST], [channel#88, id#89, sales#106, returns#107, profit#108] +(83) CometColumnarToRow [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] + +(84) HashAggregate [codegen id : 6] +Input [9]: [channel#76, id#77, spark_grouping_id#78, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90] +Keys [3]: [channel#76, id#77, spark_grouping_id#78] +Functions [3]: [sum(sales#22), sum(returns#24), sum(profit#25)] +Aggregate Attributes [3]: [sum(sales#22)#91, sum(returns#24)#92, sum(profit#25)#93] +Results [5]: [channel#76, id#77, sum(sales#22)#91 AS sales#94, sum(returns#24)#92 AS returns#95, sum(profit#25)#93 AS profit#96] + +(85) TakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#94, returns#95, profit#96] +Arguments: 100, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#94, returns#95, profit#96] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (92) -+- * CometColumnarToRow (91) - +- CometProject (90) - +- CometFilter (89) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (88) +BroadcastExchange (90) ++- * CometColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (86) -(88) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(86) 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,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] ReadSchema: struct -(89) CometFilter +(87) CometFilter Input [2]: [d_date_sk#6, d_date#7] Condition : (((isnotnull(d_date#7) AND (d_date#7 >= 2000-08-03)) AND (d_date#7 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) -(90) CometProject +(88) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(91) CometColumnarToRow [codegen id : 1] +(89) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(92) BroadcastExchange +(90) BroadcastExchange Input [1]: [d_date_sk#6] -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=9] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 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#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 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-v1_4-spark4_0/q77/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt index e15d35c438..4ce8365320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/extended.txt @@ -5,34 +5,33 @@ TakeOrderedAndProject +- HashAggregate +- Expand +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : +- CometColumnarToRow + :- 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 @@ -52,8 +51,8 @@ TakeOrderedAndProject :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -64,8 +63,8 @@ TakeOrderedAndProject : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -76,29 +75,28 @@ TakeOrderedAndProject : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - +- CometColumnarToRow + +- 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 @@ -116,4 +114,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 82 out of 109 eligible operators (75%). Final plan contains 8 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 94 out of 109 eligible operators (86%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt index 10c3b0f213..a1243769e5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -1,20 +1,20 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (8) + WholeStageCodegen (6) HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (7) + WholeStageCodegen (5) HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] Expand [sales,returns,profit,channel,id] InputAdapter Union - WholeStageCodegen (2) - Project [sales,returns,profit,profit_loss,s_store_sk] - 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 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [returns,profit,profit_loss,s_store_sk] [sales,returns,profit,channel,id] + 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] #2 CometHashAggregate [ss_ext_sales_price,ss_net_profit] [s_store_sk,sum,sum] CometProject [ss_ext_sales_price,ss_net_profit,s_store_sk] @@ -38,32 +38,28 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [s_store_sk] #5 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_sk] #7 - 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] #4 - ReusedExchange [s_store_sk] #5 - WholeStageCodegen (4) + CometBroadcastExchange [s_store_sk,returns,profit_loss] #6 + CometHashAggregate [sum,sum] [s_store_sk,returns,profit_loss,sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss))] + CometExchange [s_store_sk] #7 + 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] #4 + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (3) Project [sales,returns,profit,profit_loss,cs_call_center_sk] BroadcastNestedLoopJoin InputAdapter BroadcastExchange #8 - WholeStageCodegen (3) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + 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] #9 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] @@ -71,9 +67,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] #4 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum] [returns,profit_loss,sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss))] CometExchange #10 CometHashAggregate [cr_return_amount,cr_net_loss] [sum,sum] CometProject [cr_return_amount,cr_net_loss] @@ -81,12 +77,12 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] 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] #4 - WholeStageCodegen (6) - Project [sales,returns,profit,profit_loss,wp_web_page_sk] - 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 + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometProject [returns,profit,profit_loss,wp_web_page_sk] [sales,returns,profit,channel,id] + 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] #11 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] @@ -100,20 +96,16 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] CometBroadcastExchange [wp_web_page_sk] #12 CometFilter [wp_web_page_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page [wp_web_page_sk] - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (5) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [wp_web_page_sk] #14 - 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] #4 - ReusedExchange [wp_web_page_sk] #12 + CometBroadcastExchange [wp_web_page_sk,returns,profit_loss] #13 + 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] #14 + 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] #4 + ReusedExchange [wp_web_page_sk] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt index 532d8d869b..3d3ea5ab86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/explain.txt @@ -1,73 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (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) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- 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 (58) +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 @@ -159,263 +157,249 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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) CometColumnarToRow [codegen id : 1] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] - -(21) HashAggregate [codegen id : 1] +(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)#17, sum(UnscaledValue(ss_wholesale_cost#5))#18, sum(UnscaledValue(ss_sales_price#6))#19] -Results [6]: [d_year#13 AS ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#17 AS ss_qty#21, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#18,17,2) AS ss_wc#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#19,17,2) AS ss_sp#23] -(22) Sort [codegen id : 1] -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20 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] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(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] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(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)) -(25) CometExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(26) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) - -(29) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] - -(30) CometExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(31) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] - -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter - -(33) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) - -(34) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] - -(35) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#35, d_year#36] - -(36) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [d_date_sk#35, d_year#36] -Arguments: [ws_sold_date_sk#30], [d_date_sk#35], Inner, BuildRight - -(37) CometProject -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] - -(38) CometHashAggregate -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] - -(39) CometExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] - -(41) HashAggregate [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum(ws_quantity#27)#40, sum(UnscaledValue(ws_wholesale_cost#28))#41, sum(UnscaledValue(ws_sales_price#29))#42] -Results [6]: [d_year#36 AS ws_sold_year#43, ws_item_sk#24, ws_bill_customer_sk#25 AS ws_customer_sk#44, sum(ws_quantity#27)#40 AS ws_qty#45, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#28))#41,17,2) AS ws_wc#46, MakeDecimal(sum(UnscaledValue(ws_sales_price#29))#42,17,2) AS ws_sp#47] - -(42) Filter [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) Sort [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 3] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 3] -Output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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)) + +(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] + +(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] + +(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] + +(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 + +(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) + +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 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#56, cr_order_number#57, cr_returned_date_sk#58] +(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#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(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)) + +(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] + +(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] -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(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] -(53) CometExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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 -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(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) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(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] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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 -(58) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#59, d_year#60] +(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] -(59) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [d_date_sk#59, d_year#60] -Arguments: [cs_sold_date_sk#54], [d_date_sk#59], Inner, BuildRight +(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))] -(60) CometProject -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +(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] (61) CometHashAggregate -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] - -(62) CometExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometColumnarToRow [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] - -(64) HashAggregate [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum(cs_quantity#51)#64, sum(UnscaledValue(cs_wholesale_cost#52))#65, sum(UnscaledValue(cs_sales_price#53))#66] -Results [6]: [d_year#60 AS cs_sold_year#67, cs_item_sk#49, cs_bill_customer_sk#48 AS cs_customer_sk#68, sum(cs_quantity#51)#64 AS cs_qty#69, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#52))#65,17,2) AS cs_wc#70, MakeDecimal(sum(UnscaledValue(cs_sales_price#53))#66,17,2) AS cs_sp#71] - -(65) Filter [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(66) Sort [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST], false, 0 - -(67) SortMergeJoin [codegen id : 5] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 5] -Output [12]: [round((cast(ss_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)))), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] - -(69) TakeOrderedAndProject -Input [12]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Arguments: 100, [ratio#72 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +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))] + +(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) + +(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] + +(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 + +(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] + +(66) Project [codegen id : 1] +Output [12]: [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_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +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] + +(67) TakeOrderedAndProject +Input [12]: [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_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Arguments: 100, [ratio#63 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, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) 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 (73) -+- * CometColumnarToRow (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(70) 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 -(71) 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)) -(72) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) 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 = 23 Hosting Expression = ws_sold_date_sk#30 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#54 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-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt index 6516f29483..9d5ba53be1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/extended.txt @@ -1,63 +1,61 @@ TakeOrderedAndProject -+- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1 AS ss_sold_year#2, ss_item_sk#3, ss_customer_sk#4, sum(ss_quantity#5)#6 AS ss_qty#7, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#8))#9,17,2) AS ss_wc#10, MakeDecimal(sum(UnscaledValue(ss_sales_price#11))#12,17,2) AS ss_sp#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- Sort - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#14 AS ws_sold_year#15, ws_item_sk#16, ws_bill_customer_sk#17 AS ws_customer_sk#18, sum(ws_quantity#19)#20 AS ws_qty#21, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#22))#23,17,2) AS ws_wc#24, MakeDecimal(sum(UnscaledValue(ws_sales_price#25))#26,17,2) AS ws_sp#27)] - : +- CometColumnarToRow - : +- 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 - +- Sort - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#28 AS cs_sold_year#29, cs_item_sk#30, cs_bill_customer_sk#31 AS cs_customer_sk#32, sum(cs_quantity#33)#34 AS cs_qty#35, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#36))#37,17,2) AS cs_wc#38, MakeDecimal(sum(UnscaledValue(cs_sales_price#39))#40,17,2) AS cs_sp#41)] - +- CometColumnarToRow ++- 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 @@ -79,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 59 out of 76 eligible operators (77%). Final plan contains 4 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-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt index 761e18b0a1..77c05217f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78.native_iceberg_compat/simplified.txt @@ -1,90 +1,78 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (5) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometColumnarToRow InputAdapter - WholeStageCodegen (3) - 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 (1) - 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 - 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 - 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] - InputAdapter - WholeStageCodegen (2) - 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 - 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 - InputAdapter - WholeStageCodegen (4) - 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 - 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] + 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 + 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 [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] + 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-v1_4-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt index 532d8d869b..3d3ea5ab86 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt @@ -1,73 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (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) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- 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 (58) +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 @@ -159,263 +157,249 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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) CometColumnarToRow [codegen id : 1] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] - -(21) HashAggregate [codegen id : 1] +(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)#17, sum(UnscaledValue(ss_wholesale_cost#5))#18, sum(UnscaledValue(ss_sales_price#6))#19] -Results [6]: [d_year#13 AS ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#17 AS ss_qty#21, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#18,17,2) AS ss_wc#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#19,17,2) AS ss_sp#23] -(22) Sort [codegen id : 1] -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20 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] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(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] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(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)) -(25) CometExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(26) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) - -(29) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] - -(30) CometExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(31) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] - -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter - -(33) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) - -(34) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] - -(35) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#35, d_year#36] - -(36) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [d_date_sk#35, d_year#36] -Arguments: [ws_sold_date_sk#30], [d_date_sk#35], Inner, BuildRight - -(37) CometProject -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] - -(38) CometHashAggregate -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] - -(39) CometExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] - -(41) HashAggregate [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum(ws_quantity#27)#40, sum(UnscaledValue(ws_wholesale_cost#28))#41, sum(UnscaledValue(ws_sales_price#29))#42] -Results [6]: [d_year#36 AS ws_sold_year#43, ws_item_sk#24, ws_bill_customer_sk#25 AS ws_customer_sk#44, sum(ws_quantity#27)#40 AS ws_qty#45, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#28))#41,17,2) AS ws_wc#46, MakeDecimal(sum(UnscaledValue(ws_sales_price#29))#42,17,2) AS ws_sp#47] - -(42) Filter [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) Sort [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 3] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 3] -Output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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)) + +(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] + +(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] + +(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] + +(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 + +(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) + +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 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#56, cr_order_number#57, cr_returned_date_sk#58] +(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#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(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)) + +(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] + +(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] -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(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] -(53) CometExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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 -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(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) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(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] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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 -(58) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#59, d_year#60] +(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] -(59) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [d_date_sk#59, d_year#60] -Arguments: [cs_sold_date_sk#54], [d_date_sk#59], Inner, BuildRight +(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))] -(60) CometProject -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +(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] (61) CometHashAggregate -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] - -(62) CometExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometColumnarToRow [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] - -(64) HashAggregate [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum(cs_quantity#51)#64, sum(UnscaledValue(cs_wholesale_cost#52))#65, sum(UnscaledValue(cs_sales_price#53))#66] -Results [6]: [d_year#60 AS cs_sold_year#67, cs_item_sk#49, cs_bill_customer_sk#48 AS cs_customer_sk#68, sum(cs_quantity#51)#64 AS cs_qty#69, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#52))#65,17,2) AS cs_wc#70, MakeDecimal(sum(UnscaledValue(cs_sales_price#53))#66,17,2) AS cs_sp#71] - -(65) Filter [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(66) Sort [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST], false, 0 - -(67) SortMergeJoin [codegen id : 5] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 5] -Output [12]: [round((cast(ss_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)))), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] - -(69) TakeOrderedAndProject -Input [12]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, cs_qty#69] -Arguments: 100, [ratio#72 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, round((cast(ss_qty#21 as double) / cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)), 2) ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +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))] + +(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) + +(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] + +(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 + +(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] + +(66) Project [codegen id : 1] +Output [12]: [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_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +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] + +(67) TakeOrderedAndProject +Input [12]: [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_qty#18, ss_wc#19, ss_sp#20, ws_qty#39, cs_qty#60] +Arguments: 100, [ratio#63 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, round((cast(ss_qty#18 as double) / cast(coalesce((ws_qty#39 + cs_qty#60), 1) as double)), 2) 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 (73) -+- * CometColumnarToRow (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(70) 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 -(71) 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)) -(72) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) 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 = 23 Hosting Expression = ws_sold_date_sk#30 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#54 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-v1_4-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt index 6516f29483..9d5ba53be1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/extended.txt @@ -1,63 +1,61 @@ TakeOrderedAndProject -+- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1 AS ss_sold_year#2, ss_item_sk#3, ss_customer_sk#4, sum(ss_quantity#5)#6 AS ss_qty#7, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#8))#9,17,2) AS ss_wc#10, MakeDecimal(sum(UnscaledValue(ss_sales_price#11))#12,17,2) AS ss_sp#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- Sort - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#14 AS ws_sold_year#15, ws_item_sk#16, ws_bill_customer_sk#17 AS ws_customer_sk#18, sum(ws_quantity#19)#20 AS ws_qty#21, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#22))#23,17,2) AS ws_wc#24, MakeDecimal(sum(UnscaledValue(ws_sales_price#25))#26,17,2) AS ws_sp#27)] - : +- CometColumnarToRow - : +- 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 - +- Sort - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#28 AS cs_sold_year#29, cs_item_sk#30, cs_bill_customer_sk#31 AS cs_customer_sk#32, sum(cs_quantity#33)#34 AS cs_qty#35, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#36))#37,17,2) AS cs_wc#38, MakeDecimal(sum(UnscaledValue(cs_sales_price#39))#40,17,2) AS cs_sp#41)] - +- CometColumnarToRow ++- 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 @@ -79,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 59 out of 76 eligible operators (77%). Final plan contains 4 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-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt index 761e18b0a1..77c05217f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -1,90 +1,78 @@ TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] - WholeStageCodegen (5) + WholeStageCodegen (1) Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometColumnarToRow InputAdapter - WholeStageCodegen (3) - 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 (1) - 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 - 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 - 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] - InputAdapter - WholeStageCodegen (2) - 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 - 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 - InputAdapter - WholeStageCodegen (4) - 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 - 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] + 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 + 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 [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] + 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-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt index 65ba970881..16bf78be93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/explain.txt @@ -1,36 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- * CometColumnarToRow (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) - +- BroadcastExchange (29) - +- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +* CometColumnarToRow (31) ++- CometTakeOrderedAndProject (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- 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 (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -138,82 +137,76 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 2] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] - -(24) HashAggregate [codegen id : 2] +(23) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(25) CometFilter +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Condition : isnotnull(c_customer_sk#21) -(27) CometProject -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] +(26) CometProject +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#22, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#23, 30)) AS c_last_name#25] -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] +(27) CometBroadcastExchange +Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] -(29) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(28) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] +Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None +(29) CometProject +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -(31) Project [codegen id : 2] -Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#15] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] +(30) CometTakeOrderedAndProject +Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] -(32) TakeOrderedAndProject -Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#15] -Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24] +(31) CometColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) +BroadcastExchange (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(33) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(35) CometProject +(34) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) CometColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(37) BroadcastExchange +(36) 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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt index 62d1342926..012403275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/extended.txt @@ -1,40 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, s_city#3, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#4))#5,17,2) AS amt#6, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#8,17,2) AS profit#9)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 28 out of 35 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt index 93043d636b..8e244d6c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79.native_iceberg_compat/simplified.txt @@ -1,10 +1,10 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (2) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] @@ -35,11 +35,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, CometProject [hd_demo_sk] CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt index 65ba970881..16bf78be93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt @@ -1,36 +1,35 @@ == Physical Plan == -TakeOrderedAndProject (32) -+- * Project (31) - +- * BroadcastHashJoin Inner BuildRight (30) - :- * HashAggregate (24) - : +- * CometColumnarToRow (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) - +- BroadcastExchange (29) - +- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (25) +* CometColumnarToRow (31) ++- CometTakeOrderedAndProject (30) + +- CometProject (29) + +- CometBroadcastHashJoin (28) + :- 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 (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (24) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -138,82 +137,76 @@ Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(Unscale Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(23) CometColumnarToRow [codegen id : 2] -Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] - -(24) HashAggregate [codegen id : 2] +(23) CometHashAggregate Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15, sum#19, sum#20] Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#15] Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] -Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(26) CometFilter -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Condition : isnotnull(c_customer_sk#25) +(25) CometFilter +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Condition : isnotnull(c_customer_sk#21) -(27) CometProject -Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] -Arguments: [c_customer_sk#25, c_first_name#28, c_last_name#29], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#26, 20)) AS c_first_name#28, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#27, 30)) AS c_last_name#29] +(26) CometProject +Input [3]: [c_customer_sk#21, c_first_name#22, c_last_name#23] +Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25], [c_customer_sk#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#22, 20)) AS c_first_name#24, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#23, 30)) AS c_last_name#25] -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] +(27) CometBroadcastExchange +Input [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [c_customer_sk#21, c_first_name#24, c_last_name#25] -(29) BroadcastExchange -Input [3]: [c_customer_sk#25, c_first_name#28, c_last_name#29] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] +(28) CometBroadcastHashJoin +Left output [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27] +Right output [3]: [c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [ss_customer_sk#1], [c_customer_sk#21], Inner, BuildRight -(30) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_customer_sk#1] -Right keys [1]: [c_customer_sk#25] -Join type: Inner -Join condition: None +(29) CometProject +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#26, profit#27, c_customer_sk#21, c_first_name#24, c_last_name#25] +Arguments: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15], [c_last_name#25, c_first_name#24, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] -(31) Project [codegen id : 2] -Output [7]: [c_last_name#29, c_first_name#28, substr(s_city#15, 1, 30) AS substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#15] -Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#15, amt#23, profit#24, c_customer_sk#25, c_first_name#28, c_last_name#29] +(30) CometTakeOrderedAndProject +Input [7]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27, s_city#15] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_last_name#25 ASC NULLS FIRST,c_first_name#24 ASC NULLS FIRST,substr(s_city#15, 1, 30) ASC NULLS FIRST,profit#27 ASC NULLS FIRST], output=[c_last_name#25,c_first_name#24,substr(s_city, 1, 30)#28,ss_ticket_number#5,amt#26,profit#27]), [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27], 100, 0, [c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#27 ASC NULLS FIRST], [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] -(32) TakeOrderedAndProject -Input [7]: [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24, s_city#15] -Arguments: 100, [c_last_name#29 ASC NULLS FIRST, c_first_name#28 ASC NULLS FIRST, substr(s_city#15, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#29, c_first_name#28, substr(s_city, 1, 30)#30, ss_ticket_number#5, amt#23, profit#24] +(31) CometColumnarToRow [codegen id : 1] +Input [6]: [c_last_name#25, c_first_name#24, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#26, profit#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 -BroadcastExchange (37) -+- * CometColumnarToRow (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (33) +BroadcastExchange (36) ++- * CometColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (32) -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#10, d_year#11, d_dow#12] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] ReadSchema: struct -(34) CometFilter +(33) CometFilter Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Condition : (((isnotnull(d_dow#12) AND (d_dow#12 = 1)) AND d_year#11 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) -(35) CometProject +(34) CometProject Input [3]: [d_date_sk#10, d_year#11, d_dow#12] Arguments: [d_date_sk#10], [d_date_sk#10] -(36) CometColumnarToRow [codegen id : 1] +(35) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#10] -(37) BroadcastExchange +(36) 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=2] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt index 62d1342926..012403275a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/extended.txt @@ -1,40 +1,39 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_ticket_number#1, ss_customer_sk#2, s_city#3, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#4))#5,17,2) AS amt#6, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#8,17,2) AS profit#9)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- 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 28 out of 35 eligible operators (80%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 33 out of 35 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt index 93043d636b..8e244d6c9b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -1,10 +1,10 @@ -TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] - WholeStageCodegen (2) - Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] - BroadcastHashJoin [ss_customer_sk,c_customer_sk] - HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometProject [c_last_name,c_first_name,substr(s_city, 1, 30),ss_ticket_number,amt,profit,s_city] + CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,s_city,amt,profit,c_customer_sk,c_first_name,c_last_name] + CometHashAggregate [ss_addr_sk,sum,sum] [ss_ticket_number,ss_customer_sk,s_city,amt,profit,sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit))] CometExchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 CometHashAggregate [ss_coupon_amt,ss_net_profit] [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] CometProject [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] @@ -35,11 +35,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, CometProject [hd_demo_sk] CometFilter [hd_demo_sk,hd_dep_count,hd_vehicle_count] CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + CometBroadcastExchange [c_customer_sk,c_first_name,c_last_name] #6 + CometProject [c_first_name,c_last_name] [c_customer_sk,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt index cdaf0958a7..1b9324fe93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) +- CometProject (39) @@ -237,19 +237,17 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Input [2]: [s_store_name#9, sum#22] Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(42) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum#22] - -(43) HashAggregate [codegen id : 1] +(42) CometHashAggregate Input [2]: [s_store_name#9, sum#22] Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS sum(ss_net_profit)#24] -(44) TakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#24] -Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#24] +(43) CometTakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] + +(44) CometColumnarToRow [codegen id : 1] +Input [2]: [s_store_name#9, sum(ss_net_profit)#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt index 81edb5f8de..8e04bbeba7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_name#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS sum(ss_net_profit)#4)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -49,4 +49,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 44 out of 48 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt index 5d50f322d5..d1542ab607 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (1) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] CometExchange [s_store_name] #1 CometHashAggregate [ss_net_profit] [s_store_name,sum] CometProject [ss_net_profit,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt index cdaf0958a7..1b9324fe93 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (44) -+- * HashAggregate (43) - +- * CometColumnarToRow (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometHashAggregate (42) +- CometExchange (41) +- CometHashAggregate (40) +- CometProject (39) @@ -237,19 +237,17 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] Input [2]: [s_store_name#9, sum#22] Arguments: hashpartitioning(s_store_name#9, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(42) CometColumnarToRow [codegen id : 1] -Input [2]: [s_store_name#9, sum#22] - -(43) HashAggregate [codegen id : 1] +(42) CometHashAggregate Input [2]: [s_store_name#9, sum#22] Keys [1]: [s_store_name#9] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] -Results [2]: [s_store_name#9, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) AS sum(ss_net_profit)#24] -(44) TakeOrderedAndProject -Input [2]: [s_store_name#9, sum(ss_net_profit)#24] -Arguments: 100, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#24] +(43) CometTakeOrderedAndProject +Input [2]: [s_store_name#9, sum(ss_net_profit)#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[s_store_name#9 ASC NULLS FIRST], output=[s_store_name#9,sum(ss_net_profit)#23]), [s_store_name#9, sum(ss_net_profit)#23], 100, 0, [s_store_name#9 ASC NULLS FIRST], [s_store_name#9, sum(ss_net_profit)#23] + +(44) CometColumnarToRow [codegen id : 1] +Input [2]: [s_store_name#9, sum(ss_net_profit)#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt index 81edb5f8de..8e04bbeba7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/extended.txt @@ -1,6 +1,6 @@ -TakeOrderedAndProject -+- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_name#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS sum(ss_net_profit)#4)] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -49,4 +49,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Comet accelerated 44 out of 48 eligible operators (91%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 46 out of 48 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt index 5d50f322d5..d1542ab607 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] - WholeStageCodegen (1) - HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + CometHashAggregate [sum] [s_store_name,sum(ss_net_profit),sum(UnscaledValue(ss_net_profit))] CometExchange [s_store_name] #1 CometHashAggregate [ss_net_profit] [s_store_name,sum] CometProject [ss_net_profit,s_store_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt index 5e3755eb8c..223374746d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (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) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(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))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(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#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +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 -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(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) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 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] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(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 -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(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) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(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] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(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] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 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] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], 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 -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(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] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(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 -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(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] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(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 -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#51, 16)) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(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] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(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] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(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] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(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] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(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)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(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] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(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)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(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#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +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 -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(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)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(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 -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(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)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(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] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(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] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], 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 -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(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] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(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 -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(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] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(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 -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#83, 16)) AS web_site_id#84] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(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 -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(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 -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(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 -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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 [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +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]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 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-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt index 6379dc342e..ae6fc19b60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/extended.txt @@ -1,134 +1,131 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) AS sales#3, sum(coalesce(cast(sr_return_amt#4 as decimal(12,2)), 0.00))#5 AS returns#6, sum((ss_net_profit#7 - coalesce(cast(sr_net_loss#8 as decimal(12,2)), 0.00)))#9 AS profit#10, store channel AS channel#11, concat(store, s_store_id#12) AS id#13)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#15,17,2) AS sales#16, sum(coalesce(cast(cr_return_amount#17 as decimal(12,2)), 0.00))#18 AS returns#19, sum((cs_net_profit#20 - coalesce(cast(cr_net_loss#21 as decimal(12,2)), 0.00)))#22 AS profit#23, catalog channel AS channel#24, concat(catalog_page, cp_catalog_page_id#25) AS id#26)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#28,17,2) AS sales#29, sum(coalesce(cast(wr_return_amt#30 as decimal(12,2)), 0.00))#31 AS returns#32, sum((ws_net_profit#33 - coalesce(cast(wr_net_loss#34 as decimal(12,2)), 0.00)))#35 AS profit#36, web channel AS channel#37, concat(web_site, web_site_id#38) AS id#39)] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 115 out of 127 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80.native_iceberg_compat/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - 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] #3 - 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 #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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - 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] #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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - 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] #9 - 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 (2) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - 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] #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] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - 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] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - 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] #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] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - 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] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,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))),s_store_id] + CometExchange [s_store_id] #2 + 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] #3 + 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 #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] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + 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] #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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + 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] #9 + 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] [sales,returns,profit,channel,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))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + 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] #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] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + 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] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,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))),web_site_id] + CometExchange [web_site_id] #14 + 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] #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] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + 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] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt index 5e3755eb8c..223374746d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (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) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(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))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(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#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +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 -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(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) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 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] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(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 -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(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) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(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] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(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] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 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] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], 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 -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(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] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(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 -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(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] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(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 -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#51, 16)) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(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] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(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] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(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] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(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] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(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)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(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] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(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)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(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#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +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 -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(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)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(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 -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(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)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(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] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(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] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], 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 -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(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] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(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 -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(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] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(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 -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#83, 16)) AS web_site_id#84] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(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 -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(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 -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(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 -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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 [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +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]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 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-v1_4-spark4_0/q80/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt index 6379dc342e..ae6fc19b60 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/extended.txt @@ -1,134 +1,131 @@ -TakeOrderedAndProject -+- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#1))#2,17,2) AS sales#3, sum(coalesce(cast(sr_return_amt#4 as decimal(12,2)), 0.00))#5 AS returns#6, sum((ss_net_profit#7 - coalesce(cast(sr_net_loss#8 as decimal(12,2)), 0.00)))#9 AS profit#10, store channel AS channel#11, concat(store, s_store_id#12) AS id#13)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#15,17,2) AS sales#16, sum(coalesce(cast(cr_return_amount#17 as decimal(12,2)), 0.00))#18 AS returns#19, sum((cs_net_profit#20 - coalesce(cast(cr_net_loss#21 as decimal(12,2)), 0.00)))#22 AS profit#23, catalog channel AS channel#24, concat(catalog_page, cp_catalog_page_id#25) AS id#26)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#27))#28,17,2) AS sales#29, sum(coalesce(cast(wr_return_amt#30 as decimal(12,2)), 0.00))#31 AS returns#32, sum((ws_net_profit#33 - coalesce(cast(wr_net_loss#34 as decimal(12,2)), 0.00)))#35 AS profit#36, web channel AS channel#37, concat(web_site, web_site_id#38) AS id#39)] - +- CometColumnarToRow - +- 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 +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometSortMergeJoin + : : : : : :- CometSort + : : : : : : +- CometExchange + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : : +- SubqueryBroadcast + : : : : : : +- BroadcastExchange + : : : : : : +- 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 115 out of 127 eligible operators (90%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 123 out of 127 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_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,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - 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] #3 - 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 #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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - 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] #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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - 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] #9 - 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 (2) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - 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] #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] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - 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] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - 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))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - 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] #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] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - 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] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,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))),s_store_id] + CometExchange [s_store_id] #2 + 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] #3 + 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 #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] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + 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] #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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + 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] #9 + 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] [sales,returns,profit,channel,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))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + 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] #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] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + 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] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,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))),web_site_id] + CometExchange [web_site_id] #14 + 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] #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] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + 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] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt index 157af58b46..492a321f97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/explain.txt @@ -1,58 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (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.customer_address (9) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometExchange (29) - : : +- CometHashAggregate (28) - : : +- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (47) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (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_returns (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.customer_address (9) + : : +- CometBroadcastExchange (34) + : : +- CometFilter (33) + : : +- CometHashAggregate (32) + : : +- CometExchange (31) + : : +- CometHashAggregate (30) + : : +- CometHashAggregate (29) + : : +- CometExchange (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (40) + : +- CometProject (39) + : +- CometFilter (38) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns @@ -132,214 +128,191 @@ Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 5] -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] - -(18) HashAggregate [codegen id : 5] +(17) CometHashAggregate Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] Keys [2]: [cr_returning_customer_sk#1, ca_state#10] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#10 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(19) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) +(18) CometFilter +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(cr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Condition : isnotnull(cr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#21] -Arguments: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18], [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] - -(25) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#22, ca_state#23] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] -Right output [2]: [ca_address_sk#22, ca_state#23] -Arguments: [cr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight - -(27) CometProject -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#22, ca_state#23] -Arguments: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23], [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] - -(28) CometHashAggregate -Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] -Keys [2]: [cr_returning_customer_sk#16, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] - -(29) CometExchange -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] -Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] - -(31) HashAggregate [codegen id : 1] -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] -Keys [2]: [cr_returning_customer_sk#16, ca_state#23] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] -Results [2]: [ca_state#23 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 1] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(34) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +(20) CometFilter +Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Condition : isnotnull(cr_returning_addr_sk#16) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] + +(22) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight + +(23) CometProject +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] +Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] + +(24) ReusedExchange [Reuses operator id: 12] +Output [2]: [ca_address_sk#21, ca_state#22] + +(25) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] +Right output [2]: [ca_address_sk#21, ca_state#22] +Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight + +(26) CometProject +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] +Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] + +(27) CometHashAggregate +Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] + +(28) CometExchange +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] +Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] + +(30) CometHashAggregate +Input [2]: [ctr_state#24, ctr_total_return#25] +Keys [1]: [ctr_state#24] +Functions [1]: [partial_avg(ctr_total_return#25)] + +(31) CometExchange +Input [3]: [ctr_state#24, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(32) CometHashAggregate +Input [3]: [ctr_state#24, sum#26, count#27] +Keys [1]: [ctr_state#24] +Functions [1]: [avg(ctr_total_return#25)] + +(33) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) + +(34) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] + +(35) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight + +(36) CometProject +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(38) CometFilter +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) -(42) CometProject -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#39, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#42] +(39) CometProject +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#35, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#38] -(43) CometColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] +(40) CometBroadcastExchange +Input [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -(44) BroadcastExchange -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Right output [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 5] -Output [6]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] +(42) CometProject +Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] ReadSchema: struct -(48) CometFilter -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Condition : ((isnotnull(ca_state#50) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) = GA)) AND isnotnull(ca_address_sk#43)) +(44) CometFilter +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Condition : ((isnotnull(ca_state#46) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) = GA)) AND isnotnull(ca_address_sk#39)) -(49) CometProject -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Arguments: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60], [ca_address_sk#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#44, 10)) AS ca_street_number#55, ca_street_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#46, 15)) AS ca_street_type#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#47, 10)) AS ca_suite_number#57, ca_city#48, ca_county#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) AS ca_state#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#51, 10)) AS ca_zip#59, ca_country#52, ca_gmt_offset#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#54, 20)) AS ca_location_type#60] +(45) CometProject +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56], [ca_address_sk#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#40, 10)) AS ca_street_number#51, ca_street_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#42, 15)) AS ca_street_type#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#43, 10)) AS ca_suite_number#53, ca_city#44, ca_county#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) AS ca_state#54, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#47, 10)) AS ca_zip#55, ca_country#48, ca_gmt_offset#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#50, 20)) AS ca_location_type#56] -(50) CometColumnarToRow [codegen id : 4] -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] +(46) CometBroadcastExchange +Input [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -(51) BroadcastExchange -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(47) CometBroadcastHashJoin +Left output [6]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Right output [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [c_current_addr_sk#31], [ca_address_sk#39], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#43] -Join type: Inner -Join condition: None +(48) CometProject +Input [18]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -(53) Project [codegen id : 5] -Output [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Input [18]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] +(49) CometTakeOrderedAndProject +Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,ca_street_number#51 ASC NULLS FIRST,ca_street_name#41 ASC NULLS FIRST,ca_street_type#52 ASC NULLS FIRST,ca_suite_number#53 ASC NULLS FIRST,ca_city#44 ASC NULLS FIRST,ca_county#45 ASC NULLS FIRST,ca_state#54 ASC NULLS FIRST,ca_zip#55 ASC NULLS FIRST,ca_country#48 ASC NULLS FIRST,ca_gmt_offset#49 ASC NULLS FIRST,ca_location_type#56 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,ca_street_number#51,ca_street_name#41,ca_street_type#52,ca_suite_number#53,ca_city#44,ca_county#45,ca_state#54,ca_zip#55,ca_country#48,ca_gmt_offset#49,ca_location_type#56,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, ca_street_number#51 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#52 ASC NULLS FIRST, ca_suite_number#53 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#54 ASC NULLS FIRST, ca_zip#55 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -(54) TakeOrderedAndProject -Input [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#55 ASC NULLS FIRST, ca_street_name#45 ASC NULLS FIRST, ca_street_type#56 ASC NULLS FIRST, ca_suite_number#57 ASC NULLS FIRST, ca_city#48 ASC NULLS FIRST, ca_county#49 ASC NULLS FIRST, ca_state#58 ASC NULLS FIRST, ca_zip#59 ASC NULLS FIRST, ca_country#52 ASC NULLS FIRST, ca_gmt_offset#53 ASC NULLS FIRST, ca_location_type#60 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] +(50) CometColumnarToRow [codegen id : 1] +Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(56) CometFilter +(52) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(57) CometProject +(53) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(59) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt index 10a85ab93d..9708802e79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/extended.txt @@ -1,69 +1,65 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cr_returning_customer_sk#1 AS ctr_customer_sk#2, ca_state#3 AS ctr_state#4, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- 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.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_state#8 AS ctr_state#9, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 42 out of 61 eligible operators (68%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt index 2d929eed96..1091e272cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81.native_iceberg_compat/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (5) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #1 CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] @@ -34,43 +34,28 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st 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 #5 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #6 - WholeStageCodegen (1) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] + CometExchange [ctr_state] #6 + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometExchange [cr_returning_customer_sk,ca_state] #7 + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] #9 + CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt index 157af58b46..492a321f97 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt @@ -1,58 +1,54 @@ == Physical Plan == -TakeOrderedAndProject (54) -+- * Project (53) - +- * BroadcastHashJoin Inner BuildRight (52) - :- * Project (46) - : +- * BroadcastHashJoin Inner BuildRight (45) - : :- * Project (39) - : : +- * BroadcastHashJoin Inner BuildRight (38) - : : :- * Filter (19) - : : : +- * HashAggregate (18) - : : : +- * CometColumnarToRow (17) - : : : +- CometExchange (16) - : : : +- CometHashAggregate (15) - : : : +- CometProject (14) - : : : +- CometBroadcastHashJoin (13) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (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.customer_address (9) - : : +- BroadcastExchange (37) - : : +- * Filter (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometColumnarExchange (33) - : : +- * HashAggregate (32) - : : +- * HashAggregate (31) - : : +- * CometColumnarToRow (30) - : : +- CometExchange (29) - : : +- CometHashAggregate (28) - : : +- CometProject (27) - : : +- CometBroadcastHashJoin (26) - : : :- CometProject (24) - : : : +- CometBroadcastHashJoin (23) - : : : :- CometFilter (21) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (20) - : : : +- ReusedExchange (22) - : : +- ReusedExchange (25) - : +- BroadcastExchange (44) - : +- * CometColumnarToRow (43) - : +- CometProject (42) - : +- CometFilter (41) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (40) - +- BroadcastExchange (51) - +- * CometColumnarToRow (50) - +- CometProject (49) - +- CometFilter (48) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (47) +* CometColumnarToRow (50) ++- CometTakeOrderedAndProject (49) + +- CometProject (48) + +- CometBroadcastHashJoin (47) + :- CometProject (42) + : +- CometBroadcastHashJoin (41) + : :- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (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_returns (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.customer_address (9) + : : +- CometBroadcastExchange (34) + : : +- CometFilter (33) + : : +- CometHashAggregate (32) + : : +- CometExchange (31) + : : +- CometHashAggregate (30) + : : +- CometHashAggregate (29) + : : +- CometExchange (28) + : : +- CometHashAggregate (27) + : : +- CometProject (26) + : : +- CometBroadcastHashJoin (25) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometFilter (20) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (19) + : : : +- ReusedExchange (21) + : : +- ReusedExchange (24) + : +- CometBroadcastExchange (40) + : +- CometProject (39) + : +- CometFilter (38) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (37) + +- CometBroadcastExchange (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns @@ -132,214 +128,191 @@ Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(17) CometColumnarToRow [codegen id : 5] -Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] - -(18) HashAggregate [codegen id : 5] +(17) CometHashAggregate Input [3]: [cr_returning_customer_sk#1, ca_state#10, sum#11] Keys [2]: [cr_returning_customer_sk#1, ca_state#10] Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#12] -Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#13, ca_state#10 AS ctr_state#14, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#12,17,2) AS ctr_total_return#15] -(19) Filter [codegen id : 5] -Input [3]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15] -Condition : isnotnull(ctr_total_return#15) +(18) CometFilter +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#19), dynamicpruningexpression(cr_returned_date_sk#19 IN dynamicpruning#20)] +PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] PushedFilters: [IsNotNull(cr_returning_addr_sk)] ReadSchema: struct -(21) CometFilter -Input [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Condition : isnotnull(cr_returning_addr_sk#17) - -(22) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#21] - -(23) CometBroadcastHashJoin -Left output [4]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19] -Right output [1]: [d_date_sk#21] -Arguments: [cr_returned_date_sk#19], [d_date_sk#21], Inner, BuildRight - -(24) CometProject -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, cr_returned_date_sk#19, d_date_sk#21] -Arguments: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18], [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] - -(25) ReusedExchange [Reuses operator id: 12] -Output [2]: [ca_address_sk#22, ca_state#23] - -(26) CometBroadcastHashJoin -Left output [3]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18] -Right output [2]: [ca_address_sk#22, ca_state#23] -Arguments: [cr_returning_addr_sk#17], [ca_address_sk#22], Inner, BuildRight - -(27) CometProject -Input [5]: [cr_returning_customer_sk#16, cr_returning_addr_sk#17, cr_return_amt_inc_tax#18, ca_address_sk#22, ca_state#23] -Arguments: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23], [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] - -(28) CometHashAggregate -Input [3]: [cr_returning_customer_sk#16, cr_return_amt_inc_tax#18, ca_state#23] -Keys [2]: [cr_returning_customer_sk#16, ca_state#23] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#18))] - -(29) CometExchange -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] -Arguments: hashpartitioning(cr_returning_customer_sk#16, ca_state#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 1] -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] - -(31) HashAggregate [codegen id : 1] -Input [3]: [cr_returning_customer_sk#16, ca_state#23, sum#24] -Keys [2]: [cr_returning_customer_sk#16, ca_state#23] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#18))#12] -Results [2]: [ca_state#23 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#18))#12,17,2) AS ctr_total_return#26] - -(32) HashAggregate [codegen id : 1] -Input [2]: [ctr_state#25, ctr_total_return#26] -Keys [1]: [ctr_state#25] -Functions [1]: [partial_avg(ctr_total_return#26)] -Aggregate Attributes [2]: [sum#27, count#28] -Results [3]: [ctr_state#25, sum#29, count#30] - -(33) CometColumnarExchange -Input [3]: [ctr_state#25, sum#29, count#30] -Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] - -(34) CometColumnarToRow [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] - -(35) HashAggregate [codegen id : 2] -Input [3]: [ctr_state#25, sum#29, count#30] -Keys [1]: [ctr_state#25] -Functions [1]: [avg(ctr_total_return#26)] -Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] -Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(36) Filter [codegen id : 2] -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) - -(37) BroadcastExchange -Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] - -(38) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_state#14] -Right keys [1]: [ctr_state#25] -Join type: Inner -Join condition: (cast(ctr_total_return#15 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) - -(39) Project [codegen id : 5] -Output [2]: [ctr_customer_sk#13, ctr_total_return#15] -Input [5]: [ctr_customer_sk#13, ctr_state#14, ctr_total_return#15, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] - -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +(20) CometFilter +Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Condition : isnotnull(cr_returning_addr_sk#16) + +(21) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#20] + +(22) CometBroadcastHashJoin +Left output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Right output [1]: [d_date_sk#20] +Arguments: [cr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight + +(23) CometProject +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] +Arguments: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17], [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] + +(24) ReusedExchange [Reuses operator id: 12] +Output [2]: [ca_address_sk#21, ca_state#22] + +(25) CometBroadcastHashJoin +Left output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] +Right output [2]: [ca_address_sk#21, ca_state#22] +Arguments: [cr_returning_addr_sk#16], [ca_address_sk#21], Inner, BuildRight + +(26) CometProject +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] +Arguments: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22], [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] + +(27) CometHashAggregate +Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] + +(28) CometExchange +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] +Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(29) CometHashAggregate +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#23] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] + +(30) CometHashAggregate +Input [2]: [ctr_state#24, ctr_total_return#25] +Keys [1]: [ctr_state#24] +Functions [1]: [partial_avg(ctr_total_return#25)] + +(31) CometExchange +Input [3]: [ctr_state#24, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] + +(32) CometHashAggregate +Input [3]: [ctr_state#24, sum#26, count#27] +Keys [1]: [ctr_state#24] +Functions [1]: [avg(ctr_total_return#25)] + +(33) CometFilter +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) + +(34) CometBroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] + +(35) CometBroadcastHashJoin +Left output [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Right output [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_state#13], [ctr_state#24], Inner, (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28), BuildRight + +(36) CometProject +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#28, ctr_state#24] +Arguments: [ctr_customer_sk#12, ctr_total_return#14], [ctr_customer_sk#12, ctr_total_return#14] + +(37) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(41) CometFilter -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) +(38) CometFilter +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#31)) -(42) CometProject -Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] -Arguments: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42], [c_customer_sk#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#34, 16)) AS c_customer_id#39, c_current_addr_sk#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#36, 10)) AS c_salutation#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#42] +(39) CometProject +Input [6]: [c_customer_sk#29, c_customer_id#30, c_current_addr_sk#31, c_salutation#32, c_first_name#33, c_last_name#34] +Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [c_customer_sk#29, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#30, 16)) AS c_customer_id#35, c_current_addr_sk#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_salutation#32, 10)) AS c_salutation#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#33, 20)) AS c_first_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#34, 30)) AS c_last_name#38] -(43) CometColumnarToRow [codegen id : 3] -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] +(40) CometBroadcastExchange +Input [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -(44) BroadcastExchange -Input [6]: [c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Right output [6]: [c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [ctr_customer_sk#12], [c_customer_sk#29], Inner, BuildRight -(45) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ctr_customer_sk#13] -Right keys [1]: [c_customer_sk#33] -Join type: Inner -Join condition: None - -(46) Project [codegen id : 5] -Output [6]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] -Input [8]: [ctr_customer_sk#13, ctr_total_return#15, c_customer_sk#33, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42] +(42) CometProject +Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#29, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38], [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_state), IsNotNull(ca_address_sk)] ReadSchema: struct -(48) CometFilter -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Condition : ((isnotnull(ca_state#50) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) = GA)) AND isnotnull(ca_address_sk#43)) +(44) CometFilter +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Condition : ((isnotnull(ca_state#46) AND (static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) = GA)) AND isnotnull(ca_address_sk#39)) -(49) CometProject -Input [12]: [ca_address_sk#43, ca_street_number#44, ca_street_name#45, ca_street_type#46, ca_suite_number#47, ca_city#48, ca_county#49, ca_state#50, ca_zip#51, ca_country#52, ca_gmt_offset#53, ca_location_type#54] -Arguments: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60], [ca_address_sk#43, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#44, 10)) AS ca_street_number#55, ca_street_name#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#46, 15)) AS ca_street_type#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#47, 10)) AS ca_suite_number#57, ca_city#48, ca_county#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#50, 2)) AS ca_state#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#51, 10)) AS ca_zip#59, ca_country#52, ca_gmt_offset#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#54, 20)) AS ca_location_type#60] +(45) CometProject +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56], [ca_address_sk#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#40, 10)) AS ca_street_number#51, ca_street_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_type#42, 15)) AS ca_street_type#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_suite_number#43, 10)) AS ca_suite_number#53, ca_city#44, ca_county#45, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_state#46, 2)) AS ca_state#54, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#47, 10)) AS ca_zip#55, ca_country#48, ca_gmt_offset#49, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_location_type#50, 20)) AS ca_location_type#56] -(50) CometColumnarToRow [codegen id : 4] -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] +(46) CometBroadcastExchange +Input [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] -(51) BroadcastExchange -Input [12]: [ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +(47) CometBroadcastHashJoin +Left output [6]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38] +Right output [12]: [ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [c_current_addr_sk#31], [ca_address_sk#39], Inner, BuildRight -(52) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [c_current_addr_sk#35] -Right keys [1]: [ca_address_sk#43] -Join type: Inner -Join condition: None +(48) CometProject +Input [18]: [ctr_total_return#14, c_customer_id#35, c_current_addr_sk#31, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#39, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56] +Arguments: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -(53) Project [codegen id : 5] -Output [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Input [18]: [ctr_total_return#15, c_customer_id#39, c_current_addr_sk#35, c_salutation#40, c_first_name#41, c_last_name#42, ca_address_sk#43, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60] +(49) CometTakeOrderedAndProject +Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#35 ASC NULLS FIRST,c_salutation#36 ASC NULLS FIRST,c_first_name#37 ASC NULLS FIRST,c_last_name#38 ASC NULLS FIRST,ca_street_number#51 ASC NULLS FIRST,ca_street_name#41 ASC NULLS FIRST,ca_street_type#52 ASC NULLS FIRST,ca_suite_number#53 ASC NULLS FIRST,ca_city#44 ASC NULLS FIRST,ca_county#45 ASC NULLS FIRST,ca_state#54 ASC NULLS FIRST,ca_zip#55 ASC NULLS FIRST,ca_country#48 ASC NULLS FIRST,ca_gmt_offset#49 ASC NULLS FIRST,ca_location_type#56 ASC NULLS FIRST,ctr_total_return#14 ASC NULLS FIRST], output=[c_customer_id#35,c_salutation#36,c_first_name#37,c_last_name#38,ca_street_number#51,ca_street_name#41,ca_street_type#52,ca_suite_number#53,ca_city#44,ca_county#45,ca_state#54,ca_zip#55,ca_country#48,ca_gmt_offset#49,ca_location_type#56,ctr_total_return#14]), [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14], 100, 0, [c_customer_id#35 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, ca_street_number#51 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#52 ASC NULLS FIRST, ca_suite_number#53 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#54 ASC NULLS FIRST, ca_zip#55 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#56 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] -(54) TakeOrderedAndProject -Input [16]: [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] -Arguments: 100, [c_customer_id#39 ASC NULLS FIRST, c_salutation#40 ASC NULLS FIRST, c_first_name#41 ASC NULLS FIRST, c_last_name#42 ASC NULLS FIRST, ca_street_number#55 ASC NULLS FIRST, ca_street_name#45 ASC NULLS FIRST, ca_street_type#56 ASC NULLS FIRST, ca_suite_number#57 ASC NULLS FIRST, ca_city#48 ASC NULLS FIRST, ca_county#49 ASC NULLS FIRST, ca_state#58 ASC NULLS FIRST, ca_zip#59 ASC NULLS FIRST, ca_country#52 ASC NULLS FIRST, ca_gmt_offset#53 ASC NULLS FIRST, ca_location_type#60 ASC NULLS FIRST, ctr_total_return#15 ASC NULLS FIRST], [c_customer_id#39, c_salutation#40, c_first_name#41, c_last_name#42, ca_street_number#55, ca_street_name#45, ca_street_type#56, ca_suite_number#57, ca_city#48, ca_county#49, ca_state#58, ca_zip#59, ca_country#52, ca_gmt_offset#53, ca_location_type#60, ctr_total_return#15] +(50) CometColumnarToRow [codegen id : 1] +Input [16]: [c_customer_id#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#51, ca_street_name#41, ca_street_type#52, ca_suite_number#53, ca_city#44, ca_county#45, ca_state#54, ca_zip#55, ca_country#48, ca_gmt_offset#49, ca_location_type#56, ctr_total_return#14] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (59) -+- * CometColumnarToRow (58) - +- CometProject (57) - +- CometFilter (56) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (55) +BroadcastExchange (55) ++- * CometColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(55) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#6, d_year#7] 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 -(56) CometFilter +(52) CometFilter Input [2]: [d_date_sk#6, d_year#7] Condition : ((isnotnull(d_year#7) AND (d_year#7 = 2000)) AND isnotnull(d_date_sk#6)) -(57) CometProject +(53) CometProject Input [2]: [d_date_sk#6, d_year#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(58) CometColumnarToRow [codegen id : 1] +(54) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(59) BroadcastExchange +(55) BroadcastExchange Input [1]: [d_date_sk#6] -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=4] -Subquery:2 Hosting operator id = 20 Hosting Expression = cr_returned_date_sk#19 IN dynamicpruning#5 +Subquery:2 Hosting operator id = 19 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt index 10a85ab93d..9708802e79 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/extended.txt @@ -1,69 +1,65 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cr_returning_customer_sk#1 AS ctr_customer_sk#2, ca_state#3 AS ctr_state#4, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#5))#6,17,2) AS ctr_total_return#7)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- 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.customer_address - : : +- BroadcastExchange - : : +- Filter - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ca_state#8 AS ctr_state#9, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#10))#6,17,2) AS ctr_total_return#11)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer_address + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Comet accelerated 42 out of 61 eligible operators (68%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt index 2d929eed96..1091e272cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -1,15 +1,15 @@ -TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - WholeStageCodegen (5) - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - BroadcastHashJoin [ctr_customer_sk,c_customer_sk] - Project [ctr_customer_sk,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] - Filter [ctr_total_return] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] + CometBroadcastHashJoin [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometProject [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastHashJoin [ctr_customer_sk,ctr_total_return,c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometProject [ctr_customer_sk,ctr_total_return] + CometBroadcastHashJoin [ctr_customer_sk,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2),ctr_state] + CometFilter [ctr_customer_sk,ctr_state,ctr_total_return] + CometHashAggregate [sum] [ctr_customer_sk,ctr_state,ctr_total_return,cr_returning_customer_sk,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] CometExchange [cr_returning_customer_sk,ca_state] #1 CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] @@ -34,43 +34,28 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st 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 #5 - WholeStageCodegen (2) - Filter [(avg(ctr_total_return) * 1.2)] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] - CometColumnarToRow - InputAdapter - CometColumnarExchange [ctr_state] #6 - WholeStageCodegen (1) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] - CometColumnarToRow - InputAdapter - CometExchange [cr_returning_customer_sk,ca_state] #7 - CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] - CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] - CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] - CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #3 - ReusedExchange [ca_address_sk,ca_state] #4 - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometBroadcastExchange [(avg(ctr_total_return) * 1.2),ctr_state] #5 + CometFilter [(avg(ctr_total_return) * 1.2),ctr_state] + CometHashAggregate [sum,count] [(avg(ctr_total_return) * 1.2),ctr_state,avg(ctr_total_return)] + CometExchange [ctr_state] #6 + CometHashAggregate [ctr_total_return] [ctr_state,sum,count] + CometHashAggregate [cr_returning_customer_sk,sum] [ctr_state,ctr_total_return,ca_state,sum(UnscaledValue(cr_return_amt_inc_tax))] + CometExchange [cr_returning_customer_sk,ca_state] #7 + CometHashAggregate [cr_return_amt_inc_tax] [cr_returning_customer_sk,ca_state,sum] + CometProject [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,ca_address_sk,ca_state] + CometProject [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + CometBroadcastHashJoin [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk,d_date_sk] + CometFilter [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #3 + ReusedExchange [ca_address_sk,ca_state] #4 + CometBroadcastExchange [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] #8 + CometProject [c_customer_id,c_salutation,c_first_name,c_last_name] [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometFilter [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + CometBroadcastExchange [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] #9 + CometProject [ca_street_number,ca_street_type,ca_suite_number,ca_state,ca_zip,ca_location_type] [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometFilter [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt index de788a42e2..54520c592a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt @@ -1,29 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * CometColumnarToRow (22) - +- CometSort (21) - +- CometColumnarExchange (20) - +- * HashAggregate (19) - +- * CometColumnarToRow (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (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) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometExchange (19) + +- CometHashAggregate (18) + +- CometExchange (17) + +- CometHashAggregate (16) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -107,68 +106,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] - -(19) HashAggregate [codegen id : 1] +(18) CometHashAggregate Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#16] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS total_sum#17, i_category#12, i_class#13, (cast((shiftright(spark_grouping_id#14, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint)) AS lochierarchy#18, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS _w0#19, (cast((shiftright(spark_grouping_id#14, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint) = 0) THEN i_category#12 END AS _w2#21] -(20) CometColumnarExchange -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(19) CometExchange +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(21) CometSort -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] +(20) CometSort +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(22) CometColumnarToRow [codegen id : 2] -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +(21) CometColumnarToRow [codegen id : 1] +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -(23) Window -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] +(22) Window +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(24) Project [codegen id : 3] -Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] +(23) Project [codegen id : 2] +Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -(25) TakeOrderedAndProject -Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(24) TakeOrderedAndProject +Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(25) 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 -(27) CometFilter +(26) 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)) -(28) CometProject +(27) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(30) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#5] 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-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt index 921abcd329..dfc810b108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt @@ -3,31 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt index 539b436013..29e2d72920 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt @@ -1,39 +1,36 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - 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 #3 - 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] #4 - 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] #5 - 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] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + 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 #3 + 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] #4 + 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] #5 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index de788a42e2..54520c592a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -1,29 +1,28 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * CometColumnarToRow (22) - +- CometSort (21) - +- CometColumnarExchange (20) - +- * HashAggregate (19) - +- * CometColumnarToRow (18) - +- CometExchange (17) - +- CometHashAggregate (16) - +- CometExpand (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) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometExchange (19) + +- CometHashAggregate (18) + +- CometExchange (17) + +- CometHashAggregate (16) + +- CometExpand (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) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -107,68 +106,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] Arguments: hashpartitioning(i_category#12, i_class#13, spark_grouping_id#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(18) CometColumnarToRow [codegen id : 1] -Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] - -(19) HashAggregate [codegen id : 1] +(18) CometHashAggregate Input [4]: [i_category#12, i_class#13, spark_grouping_id#14, sum#15] Keys [3]: [i_category#12, i_class#13, spark_grouping_id#14] Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#16] -Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS total_sum#17, i_category#12, i_class#13, (cast((shiftright(spark_grouping_id#14, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint)) AS lochierarchy#18, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#16,17,2) AS _w0#19, (cast((shiftright(spark_grouping_id#14, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint)) AS _w1#20, CASE WHEN (cast((shiftright(spark_grouping_id#14, 0) & 1) as tinyint) = 0) THEN i_category#12 END AS _w2#21] -(20) CometColumnarExchange -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(19) CometExchange +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(21) CometSort -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] +(20) CometSort +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(22) CometColumnarToRow [codegen id : 2] -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +(21) CometColumnarToRow [codegen id : 1] +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -(23) Window -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] +(22) Window +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(24) Project [codegen id : 3] -Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] +(23) Project [codegen id : 2] +Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -(25) TakeOrderedAndProject -Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(24) TakeOrderedAndProject +Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== 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 (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(25) 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 -(27) CometFilter +(26) 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)) -(28) CometProject +(27) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(29) CometColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(30) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#5] 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-v1_4-spark4_0/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt index 921abcd329..dfc810b108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt @@ -3,31 +3,30 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- 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 + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- 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 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index 539b436013..29e2d72920 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -1,39 +1,36 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [_w0,_w1,_w2] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - 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 #3 - 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] #4 - 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] #5 - 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] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + 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 #3 + 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] #4 + 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] #5 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt index 974bebdf5e..185c9d264c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#10))#20] -Results [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] +(24) CometSort +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] +(26) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(28) Filter [codegen id : 3] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END +(27) Filter [codegen id : 2] +Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(29) Project [codegen id : 3] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] +(28) Project [codegen id : 2] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -(30) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +(29) TakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) 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), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_moy#15] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#13, d_moy#15] 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-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt index ac5dbbed5a..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt index 4fc9029de0..61bfd1d960 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,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 - CometProject [d_date_sk,d_moy] - 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_moy] #5 - CometProject [d_date_sk,d_moy] - 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] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,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 + CometProject [d_date_sk,d_moy] + 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_moy] #5 + CometProject [d_date_sk,d_moy] + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 974bebdf5e..185c9d264c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -1,34 +1,33 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * HashAggregate (23) - +- * CometColumnarToRow (22) - +- CometExchange (21) - +- CometHashAggregate (20) - +- CometProject (19) - +- CometBroadcastHashJoin (18) - :- CometProject (14) - : +- CometBroadcastHashJoin (13) - : :- 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 (12) - : +- CometProject (11) - : +- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - +- CometBroadcastExchange (17) - +- CometFilter (16) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) + +- CometSort (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometExchange (21) + +- CometHashAggregate (20) + +- CometProject (19) + +- CometBroadcastHashJoin (18) + :- CometProject (14) + : +- CometBroadcastHashJoin (13) + : :- 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 (12) + : +- CometProject (11) + : +- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + +- CometBroadcastExchange (17) + +- CometFilter (16) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -132,72 +131,67 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#10))] Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] Arguments: hashpartitioning(i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [7]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum#19] Keys [6]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15] Functions [1]: [sum(UnscaledValue(ss_sales_price#10))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#10))#20] -Results [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#20,17,2) AS _w0#22] -(24) CometColumnarExchange -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(23) CometExchange +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(25) CometSort -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] +(24) CometSort +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +(25) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] +(26) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(28) Filter [codegen id : 3] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] -Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END +(27) Filter [codegen id : 2] +Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] +Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(29) Project [codegen id : 3] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] +(28) Project [codegen id : 2] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -(30) TakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +(29) TakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 -BroadcastExchange (35) -+- * CometColumnarToRow (34) - +- CometProject (33) - +- CometFilter (32) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (31) +BroadcastExchange (34) ++- * CometColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (30) -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(30) 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), EqualTo(d_year,1999), IsNotNull(d_date_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((isnotnull(d_year#14) AND (d_year#14 = 1999)) AND isnotnull(d_date_sk#13)) -(33) CometProject +(32) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13, d_moy#15], [d_date_sk#13, d_moy#15] -(34) CometColumnarToRow [codegen id : 1] +(33) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_moy#15] -(35) BroadcastExchange +(34) BroadcastExchange Input [2]: [d_date_sk#13, d_moy#15] 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-v1_4-spark4_0/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt index ac5dbbed5a..825b1ed81c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt @@ -4,35 +4,34 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - +- CometColumnarToRow - +- 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 - : : +- CometProject - : : +- CometFilter - : : +- 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 + +- 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 + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 4fc9029de0..61bfd1d960 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -1,44 +1,41 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,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 - CometProject [d_date_sk,d_moy] - 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_moy] #5 - CometProject [d_date_sk,d_moy] - 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] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,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 + CometProject [d_date_sk,d_moy] + 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_moy] #5 + CometProject [d_date_sk,d_moy] + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt index 059b282c84..f4fc7165aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometSort (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * CometColumnarToRow (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) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (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.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) +* CometColumnarToRow (43) ++- CometSort (42) + +- CometExchange (41) + +- 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) + : : : : : :- CometProject (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : : +- CometBroadcastExchange (17) + : : : +- CometFilter (16) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (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.customer_demographics (26) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center @@ -230,56 +229,51 @@ Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] - -(41) HashAggregate [codegen id : 1] +(40) CometHashAggregate Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#28] -Results [4]: [cc_call_center_id#5 AS Call_Center#29, cc_name#3 AS Call_Center_Name#30, cc_manager#4 AS Manager#31, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#28,17,2) AS Returns_Loss#32] -(42) CometColumnarExchange -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] -Arguments: rangepartitioning(Returns_Loss#32 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(41) CometExchange +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(43) CometSort -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] -Arguments: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32], [Returns_Loss#32 DESC NULLS LAST] +(42) CometSort +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] +Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] -(44) CometColumnarToRow [codegen id : 2] -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] +(43) CometColumnarToRow [codegen id : 1] +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(44) 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: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(45) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) -(47) CometProject +(46) CometProject Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(48) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(49) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#11] 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-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt index 564f1287a2..a1c6b73d37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/extended.txt @@ -1,52 +1,51 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cc_call_center_id#1 AS Call_Center#2, cc_name#3 AS Call_Center_Name#4, cc_manager#5 AS Manager#6, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#8,17,2) AS Returns_Loss#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt index 99c7e48bfb..3e9b8945da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91.native_iceberg_compat/simplified.txt @@ -1,56 +1,53 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - WholeStageCodegen (1) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - CometColumnarToRow - InputAdapter - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_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 [d_date_sk] #5 - 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 [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - 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] + CometExchange [Returns_Loss] #1 + CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] + CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_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 [d_date_sk] #5 + 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 [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange [hd_demo_sk] #9 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt index 059b282c84..f4fc7165aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt @@ -1,48 +1,47 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometSort (43) - +- CometColumnarExchange (42) - +- * HashAggregate (41) - +- * CometColumnarToRow (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) - : : : : : :- CometProject (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) - : : : : : +- CometBroadcastExchange (6) - : : : : : +- CometFilter (5) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) - : : : : +- CometBroadcastExchange (12) - : : : : +- CometProject (11) - : : : : +- CometFilter (10) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : : +- CometBroadcastExchange (17) - : : : +- CometFilter (16) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (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.customer_demographics (26) - +- CometBroadcastExchange (35) - +- CometProject (34) - +- CometFilter (33) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) +* CometColumnarToRow (43) ++- CometSort (42) + +- CometExchange (41) + +- 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) + : : : : : :- CometProject (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (1) + : : : : : +- CometBroadcastExchange (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (4) + : : : : +- CometBroadcastExchange (12) + : : : : +- CometProject (11) + : : : : +- CometFilter (10) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : : +- CometBroadcastExchange (17) + : : : +- CometFilter (16) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (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.customer_demographics (26) + +- CometBroadcastExchange (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (32) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center @@ -230,56 +229,51 @@ Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#8))] Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] Arguments: hashpartitioning(cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(40) CometColumnarToRow [codegen id : 1] -Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] - -(41) HashAggregate [codegen id : 1] +(40) CometHashAggregate Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24, sum#27] Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#8))#28] -Results [4]: [cc_call_center_id#5 AS Call_Center#29, cc_name#3 AS Call_Center_Name#30, cc_manager#4 AS Manager#31, MakeDecimal(sum(UnscaledValue(cr_net_loss#8))#28,17,2) AS Returns_Loss#32] -(42) CometColumnarExchange -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] -Arguments: rangepartitioning(Returns_Loss#32 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(41) CometExchange +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(43) CometSort -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] -Arguments: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32], [Returns_Loss#32 DESC NULLS LAST] +(42) CometSort +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] +Arguments: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31], [Returns_Loss#31 DESC NULLS LAST] -(44) CometColumnarToRow [codegen id : 2] -Input [4]: [Call_Center#29, Call_Center_Name#30, Manager#31, Returns_Loss#32] +(43) CometColumnarToRow [codegen id : 1] +Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(44) 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: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(45) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((isnotnull(d_year#12) AND isnotnull(d_moy#13)) AND (d_year#12 = 1998)) AND (d_moy#13 = 11)) AND isnotnull(d_date_sk#11)) -(47) CometProject +(46) CometProject Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Arguments: [d_date_sk#11], [d_date_sk#11] -(48) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#11] -(49) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#11] 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-v1_4-spark4_0/q91/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt index 564f1287a2..a1c6b73d37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/extended.txt @@ -1,52 +1,51 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cc_call_center_id#1 AS Call_Center#2, cc_name#3 AS Call_Center_Name#4, cc_manager#5 AS Manager#6, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#8,17,2) AS Returns_Loss#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometBroadcastHashJoin - : : : : : :- CometProject - : : : : : : +- CometFilter - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : : : : : +- CometBroadcastExchange - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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.customer - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometBroadcastHashJoin + : : : : : :- CometProject + : : : : : : +- CometFilter + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : : : : : +- CometBroadcastExchange + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : : : : : +- SubqueryBroadcast + : : : : : +- BroadcastExchange + : : : : : +- 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.customer + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 45 out of 47 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt index 99c7e48bfb..3e9b8945da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -1,56 +1,53 @@ -WholeStageCodegen (2) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 - WholeStageCodegen (1) - HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] - CometColumnarToRow - InputAdapter - CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 - CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] - CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] - CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] - CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] - CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 - CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_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 [d_date_sk] #5 - 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 [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 - CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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] - CometBroadcastExchange [ca_address_sk] #7 - CometProject [ca_address_sk] - CometFilter [ca_address_sk,ca_gmt_offset] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] - CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 - CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] - CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] - CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] - CometBroadcastExchange [hd_demo_sk] #9 - 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] + CometExchange [Returns_Loss] #1 + CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] + CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status,hd_demo_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,ca_address_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + CometBroadcastHashJoin [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk,d_date_sk] + CometProject [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + CometBroadcastHashJoin [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometProject [cc_call_center_id] [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometFilter [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + CometBroadcastExchange [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] #3 + CometFilter [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_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 [d_date_sk] #5 + 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 [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] #6 + CometFilter [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_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] + CometBroadcastExchange [ca_address_sk] #7 + CometProject [ca_address_sk] + CometFilter [ca_address_sk,ca_gmt_offset] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + CometBroadcastExchange [cd_demo_sk,cd_marital_status,cd_education_status] #8 + CometProject [cd_marital_status,cd_education_status] [cd_demo_sk,cd_marital_status,cd_education_status] + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + CometBroadcastExchange [hd_demo_sk] #9 + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt index 417130e8b9..cdc8dbc3bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) +* CometColumnarToRow (30) ++- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) +- CometProject (26) @@ -163,15 +163,13 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] Input [1]: [sum#17] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#17] - -(30) HashAggregate [codegen id : 1] +(29) CometHashAggregate Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#18] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#18,17,2) AS Excess Discount Amount #19] + +(30) CometColumnarToRow [codegen id : 1] +Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== @@ -184,18 +182,18 @@ BroadcastExchange (35) (31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#20] +Output [2]: [d_date_sk#16, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#16, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 2000-01-27)) AND (d_date#20 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) (33) CometProject -Input [2]: [d_date_sk#16, d_date#20] +Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] (34) CometColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt index 3d1e721074..20df9a8047 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#1))#2,17,2) AS Excess Discount Amount #3)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 38 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt index 93926eeca5..4ec5755df7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 CometHashAggregate [ws_ext_discount_amt] [sum] CometProject [ws_ext_discount_amt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt index 417130e8b9..cdc8dbc3bc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (30) -+- * CometColumnarToRow (29) +* CometColumnarToRow (30) ++- CometHashAggregate (29) +- CometExchange (28) +- CometHashAggregate (27) +- CometProject (26) @@ -163,15 +163,13 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] Input [1]: [sum#17] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(29) CometColumnarToRow [codegen id : 1] -Input [1]: [sum#17] - -(30) HashAggregate [codegen id : 1] +(29) CometHashAggregate Input [1]: [sum#17] Keys: [] Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#18] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#18,17,2) AS Excess Discount Amount #19] + +(30) CometColumnarToRow [codegen id : 1] +Input [1]: [Excess Discount Amount #18] ===== Subqueries ===== @@ -184,18 +182,18 @@ BroadcastExchange (35) (31) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#16, d_date#20] +Output [2]: [d_date_sk#16, d_date#19] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] ReadSchema: struct (32) CometFilter -Input [2]: [d_date_sk#16, d_date#20] -Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 2000-01-27)) AND (d_date#20 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) +Input [2]: [d_date_sk#16, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 2000-01-27)) AND (d_date#19 <= 2000-04-26)) AND isnotnull(d_date_sk#16)) (33) CometProject -Input [2]: [d_date_sk#16, d_date#20] +Input [2]: [d_date_sk#16, d_date#19] Arguments: [d_date_sk#16], [d_date_sk#16] (34) CometColumnarToRow [codegen id : 1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt index 3d1e721074..20df9a8047 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#1))#2,17,2) AS Excess Discount Amount #3)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -39,4 +39,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 38 eligible operators (89%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 38 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt index 93926eeca5..4ec5755df7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (1) - HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum] [Excess Discount Amount ,sum(UnscaledValue(ws_ext_discount_amt))] CometExchange #1 CometHashAggregate [ws_ext_discount_amt] [sum] CometProject [ws_ext_discount_amt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt index 4535be2e4a..3efafa3b41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (41) -+- * CometColumnarToRow (40) +* CometColumnarToRow (41) ++- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) @@ -225,13 +225,11 @@ Results [3]: [sum#20, sum#21, count#25] Input [3]: [sum#20, sum#21, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#20, sum#21, count#25] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [count(ws_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#22,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#23,17,2) AS total net profit #28] + +(41) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt index 28a0183d43..eac4939621 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(ws_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -40,4 +40,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 36 out of 39 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt index bf85b844c9..629178d106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt index 4535be2e4a..3efafa3b41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (41) -+- * CometColumnarToRow (40) +* CometColumnarToRow (41) ++- CometHashAggregate (40) +- CometColumnarExchange (39) +- * HashAggregate (38) +- * HashAggregate (37) @@ -225,13 +225,11 @@ Results [3]: [sum#20, sum#21, count#25] Input [3]: [sum#20, sum#21, count#25] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(40) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#20, sum#21, count#25] - -(41) HashAggregate [codegen id : 2] +(40) CometHashAggregate Input [3]: [sum#20, sum#21, count#25] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#22, sum(UnscaledValue(ws_net_profit#7))#23, count(ws_order_number#5)#24] -Results [3]: [count(ws_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#22,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#23,17,2) AS total net profit #28] + +(41) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #26, total shipping cost #27, total net profit #28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt index 28a0183d43..eac4939621 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(ws_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -40,4 +40,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 36 out of 39 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 39 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt index bf85b844c9..629178d106 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt index 2ef14f1160..d86eccd434 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (54) -+- * CometColumnarToRow (53) +* CometColumnarToRow (54) ++- CometHashAggregate (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) @@ -292,13 +292,11 @@ Results [3]: [sum#25, sum#26, count#30] Input [3]: [sum#25, sum#26, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#25, sum#26, count#30] - -(54) HashAggregate [codegen id : 2] +(53) CometHashAggregate Input [3]: [sum#25, sum#26, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [count(ws_order_number#4)#29 AS order count #31, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#27,17,2) AS total shipping cost #32, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#28,17,2) AS total net profit #33] + +(54) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #31, total shipping cost #32, total net profit #33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt index fe03ddecc3..6ff8eba58f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(ws_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -62,4 +62,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt index 3e9cdb81f5..6c1b1bccfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95.native_iceberg_compat/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt index 2ef14f1160..d86eccd434 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt @@ -1,6 +1,6 @@ == Physical Plan == -* HashAggregate (54) -+- * CometColumnarToRow (53) +* CometColumnarToRow (54) ++- CometHashAggregate (53) +- CometColumnarExchange (52) +- * HashAggregate (51) +- * HashAggregate (50) @@ -292,13 +292,11 @@ Results [3]: [sum#25, sum#26, count#30] Input [3]: [sum#25, sum#26, count#30] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(53) CometColumnarToRow [codegen id : 2] -Input [3]: [sum#25, sum#26, count#30] - -(54) HashAggregate [codegen id : 2] +(53) CometHashAggregate Input [3]: [sum#25, sum#26, count#30] Keys: [] Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#27, sum(UnscaledValue(ws_net_profit#6))#28, count(ws_order_number#4)#29] -Results [3]: [count(ws_order_number#4)#29 AS order count #31, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#27,17,2) AS total shipping cost #32, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#28,17,2) AS total net profit #33] + +(54) CometColumnarToRow [codegen id : 2] +Input [3]: [order count #31, total shipping cost #32, total net profit #33] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt index fe03ddecc3..6ff8eba58f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/extended.txt @@ -1,5 +1,5 @@ - HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(count(ws_order_number#1)#2 AS order count #3, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#4))#5,17,2) AS total shipping cost #6, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#8,17,2) AS total net profit #9)] -+- CometColumnarToRow +CometColumnarToRow ++- CometHashAggregate +- CometColumnarExchange +- HashAggregate +- HashAggregate [COMET: Unsupported aggregation mode PartialMerge] @@ -62,4 +62,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 58 out of 61 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 59 out of 61 eligible operators (96%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt index 3e9cdb81f5..6c1b1bccfa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -1,7 +1,7 @@ WholeStageCodegen (2) - HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] - CometColumnarToRow - InputAdapter + CometColumnarToRow + InputAdapter + CometHashAggregate [sum,sum,count] [order count ,total shipping cost ,total net profit ,count(ws_order_number),sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit))] CometColumnarExchange #1 WholeStageCodegen (1) HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt index 3c2b54e5b2..158e7520c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,31 +1,30 @@ == Physical Plan == -* CometColumnarToRow (27) -+- CometProject (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (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.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 @@ -105,79 +104,74 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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=3] +(23) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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] -(25) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [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] +(24) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [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] -(26) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(25) CometProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(27) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(26) CometColumnarToRow [codegen id : 3] +Input [6]: [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 = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (32) -+- * CometColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) 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 -(29) CometFilter +(28) 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)) -(30) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(31) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(32) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt index aaefeae86c..4c972848e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt @@ -6,30 +6,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt index 3c91801f00..af05a7d8c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,43 +1,40 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - 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 - 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] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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-v1_4-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index 3c2b54e5b2..158e7520c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -1,31 +1,30 @@ == Physical Plan == -* CometColumnarToRow (27) -+- CometProject (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (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.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 @@ -105,79 +104,74 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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_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, i_item_id#11] -(19) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(18) CometExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(20) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] +(19) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +(20) CometColumnarToRow [codegen id : 1] +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -(24) CometColumnarExchange -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -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=3] +(23) CometColumnarExchange +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +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] -(25) CometSort -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [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] +(24) CometSort +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [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] -(26) CometProject -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(25) CometProject +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(27) CometColumnarToRow [codegen id : 4] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(26) CometColumnarToRow [codegen id : 3] +Input [6]: [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 = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (32) -+- * CometColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) 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 -(29) CometFilter +(28) 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)) -(30) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(31) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(32) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] 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-v1_4-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt index aaefeae86c..4c972848e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt @@ -6,30 +6,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- 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 + +- 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 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 3c91801f00..af05a7d8c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -1,43 +1,40 @@ -WholeStageCodegen (4) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) + WholeStageCodegen (2) Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] InputAdapter Window [_w0,i_class] - WholeStageCodegen (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - 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 - 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] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,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-spark4_0/q11.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt index d3516d1223..676d488824 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discoun 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 : 4] -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] - -(17) HashAggregate [codegen id : 4] +(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)))#23] -Results [2]: [c_customer_id#9 AS customer_id#24, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#23,18,2) AS year_total#25] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#24, year_total#25] -Condition : (isnotnull(year_total#25) AND (year_total#25 > 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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] +(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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) -(21) CometProject -Input [8]: [c_customer_sk#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Arguments: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#30, 1)) AS c_preferred_cust_flag#37, c_birth_country#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#32, 13)) AS c_login#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#33, 50)) AS c_email_address#39] +(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#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] 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#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Condition : isnotnull(ss_customer_sk#40) +(22) CometFilter +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Condition : isnotnull(ss_customer_sk#39) -(24) CometBroadcastExchange -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(25) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Right output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_sk#26], [ss_customer_sk#40], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight -(26) CometProject -Input [12]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(26) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: [d_date_sk#45, d_year#46] - -(30) CometBroadcastHashJoin -Left output [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Right output [2]: [d_date_sk#45, d_year#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#45], Inner, BuildRight - -(31) CometProject -Input [12]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43, d_date_sk#45, d_year#46] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] - -(32) CometHashAggregate -Input [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] - -(33) CometExchange -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] - -(35) HashAggregate [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23] -Results [5]: [c_customer_id#34 AS customer_id#48, c_first_name#35 AS customer_first_name#49, c_last_name#36 AS customer_last_name#50, c_email_address#39 AS customer_email_address#51, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23,18,2) AS year_total#52] - -(36) BroadcastExchange -Input [5]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#48] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] +(27) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] + +(31) CometHashAggregate +Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(32) CometExchange +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(34) CometBroadcastExchange +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(37) CometFilter +Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] +Condition : (isnotnull(c_customer_sk#52) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)))) -(40) CometProject -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Arguments: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#57, 1)) AS c_preferred_cust_flag#64, c_birth_country#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#59, 13)) AS c_login#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#60, 50)) AS c_email_address#66] +(38) CometProject +Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] +Arguments: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65], [c_customer_sk#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)) AS c_customer_id#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#54, 20)) AS c_first_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#55, 30)) AS c_last_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#56, 1)) AS c_preferred_cust_flag#63, c_birth_country#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#58, 13)) AS c_login#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#59, 50)) AS c_email_address#65] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Condition : isnotnull(ws_bill_customer_sk#67) +(40) CometFilter +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Condition : isnotnull(ws_bill_customer_sk#66) + +(41) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(43) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65] +Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_sk#52], [ws_bill_customer_sk#66], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66] -Right output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#67], Inner, BuildRight +(43) CometProject +Input [12]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(45) CometProject -Input [12]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#71, d_year#72] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#72, d_year#73] +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Right output [2]: [d_date_sk#71, d_year#72] +Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Right output [2]: [d_date_sk#72, d_year#73] -Arguments: [ws_sold_date_sk#70], [d_date_sk#72], Inner, BuildRight +(46) CometProject +Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] +Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -(48) CometProject -Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#72, d_year#73] -Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] +(47) CometHashAggregate +Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] + +(48) CometExchange +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] - -(50) CometExchange -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] -Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] - -(52) HashAggregate [codegen id : 2] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75] -Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75,18,2) AS year_total#77] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#76, year_total#77] -Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#76] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [8]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, year_total#77] -Input [9]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, customer_id#76, year_total#77] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] + +(50) CometFilter +Input [2]: [customer_id#74, year_total#75] +Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74, year_total#75] + +(52) CometBroadcastHashJoin +Left output [7]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Right output [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#23], [customer_id#74], Inner, BuildRight + +(53) CometProject +Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] +Arguments: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75], [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Condition : (isnotnull(c_customer_sk#78) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)))) +(55) CometFilter +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) -(59) CometProject -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Arguments: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91], [c_customer_sk#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)) AS c_customer_id#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#80, 20)) AS c_first_name#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#81, 30)) AS c_last_name#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#82, 1)) AS c_preferred_cust_flag#89, c_birth_country#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#84, 13)) AS c_login#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#85, 50)) AS c_email_address#91] +(56) CometProject +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Condition : isnotnull(ws_bill_customer_sk#92) +(58) CometFilter +Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Condition : isnotnull(ws_bill_customer_sk#90) + +(59) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] + +(60) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] +Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight -(62) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] +(61) CometProject +Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#95, d_year#96] (63) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91] -Right output [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [c_customer_sk#78], [ws_bill_customer_sk#92], Inner, BuildRight +Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Right output [2]: [d_date_sk#95, d_year#96] +Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight (64) CometProject -Input [12]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95], [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#97, d_year#98] - -(66) CometBroadcastHashJoin -Left output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [ws_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(67) CometProject -Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98], [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98] - -(68) CometHashAggregate -Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] - -(69) CometExchange -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] -Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] - -(71) HashAggregate [codegen id : 3] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75] -Results [2]: [c_customer_id#86 AS customer_id#100, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75,18,2) AS year_total#101] - -(72) BroadcastExchange -Input [2]: [customer_id#100, year_total#101] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#100] -Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.00) THEN (year_total#101 / year_total#77) ELSE 0E-20 END > CASE WHEN (year_total#25 > 0.00) THEN (year_total#52 / year_total#25) ELSE 0E-20 END) - -(74) Project [codegen id : 4] -Output [4]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] -Input [10]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, year_total#77, customer_id#100, year_total#101] - -(75) TakeOrderedAndProject -Input [4]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] -Arguments: 100, [customer_id#48 ASC NULLS FIRST, customer_first_name#49 ASC NULLS FIRST, customer_last_name#50 ASC NULLS FIRST, customer_email_address#51 ASC NULLS FIRST], [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] +Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] +Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] + +(65) CometHashAggregate +Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] + +(66) CometExchange +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] +Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#98, year_total#99] +Arguments: [customer_id#98, year_total#99] + +(69) CometBroadcastHashJoin +Left output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] +Right output [2]: [customer_id#98, year_total#99] +Arguments: [customer_id#23], [customer_id#98], Inner, (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#51 / year_total#24) ELSE 0E-20 END), BuildRight + +(70) CometProject +Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#98, year_total#99] +Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] + +(71) CometTakeOrderedAndProject +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_email_address#50 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_email_address#50]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] + +(72) CometColumnarToRow [codegen id : 1] +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#44 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#45, d_year#46] +(77) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) +(78) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#45, d_year#46] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#44, d_year#45] -(83) BroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#70 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#44 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#43 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt index 5f4f9a5e17..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#4)))#5,18,2) AS year_total#6)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#7 AS customer_id#8, c_first_name#9 AS customer_first_name#10, c_last_name#11 AS customer_last_name#12, c_email_address#13 AS customer_email_address#14, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#15 - ss_ext_discount_amt#16)))#5,18,2) AS year_total#17)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#18 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#20 - ws_ext_discount_amt#21)))#22,18,2) AS year_total#23)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#24 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#26 - ws_ext_discount_amt#27)))#22,18,2) AS year_total#28)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-spark4_0/q11.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt index 397a5f6da9..157d1d587e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11.native_iceberg_compat/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-spark4_0/q11/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt index d3516d1223..676d488824 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discoun 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 : 4] -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] - -(17) HashAggregate [codegen id : 4] +(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)))#23] -Results [2]: [c_customer_id#9 AS customer_id#24, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#17 - ss_ext_discount_amt#16)))#23,18,2) AS year_total#25] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#24, year_total#25] -Condition : (isnotnull(year_total#25) AND (year_total#25 > 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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] +(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#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Condition : (isnotnull(c_customer_sk#26) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)))) -(21) CometProject -Input [8]: [c_customer_sk#26, c_customer_id#27, c_first_name#28, c_last_name#29, c_preferred_cust_flag#30, c_birth_country#31, c_login#32, c_email_address#33] -Arguments: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39], [c_customer_sk#26, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#27, 16)) AS c_customer_id#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#28, 20)) AS c_first_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#29, 30)) AS c_last_name#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#30, 1)) AS c_preferred_cust_flag#37, c_birth_country#31, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#32, 13)) AS c_login#38, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#33, 50)) AS c_email_address#39] +(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#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38], [c_customer_sk#25, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#26, 16)) AS c_customer_id#33, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#27, 20)) AS c_first_name#34, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#28, 30)) AS c_last_name#35, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#29, 1)) AS c_preferred_cust_flag#36, c_birth_country#30, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#31, 13)) AS c_login#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#32, 50)) AS c_email_address#38] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] 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#42), dynamicpruningexpression(ss_sold_date_sk#42 IN dynamicpruning#43)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Condition : isnotnull(ss_customer_sk#40) +(22) CometFilter +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Condition : isnotnull(ss_customer_sk#39) -(24) CometBroadcastExchange -Input [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(23) CometBroadcastExchange +Input [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(25) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Right output [4]: [ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_sk#26], [ss_customer_sk#40], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Right output [4]: [ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_sk#25], [ss_customer_sk#39], Inner, BuildRight -(26) CometProject -Input [12]: [c_customer_sk#26, c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_customer_sk#40, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] +(25) CometProject +Input [12]: [c_customer_sk#25, c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_customer_sk#39, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#45, d_year#46] +(26) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: [d_date_sk#45, d_year#46] - -(30) CometBroadcastHashJoin -Left output [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43] -Right output [2]: [d_date_sk#45, d_year#46] -Arguments: [ss_sold_date_sk#43], [d_date_sk#45], Inner, BuildRight - -(31) CometProject -Input [12]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, ss_sold_date_sk#43, d_date_sk#45, d_year#46] -Arguments: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46], [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] - -(32) CometHashAggregate -Input [10]: [c_customer_id#34, c_first_name#35, c_last_name#36, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, ss_ext_discount_amt#41, ss_ext_list_price#42, d_year#46] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] - -(33) CometExchange -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Arguments: hashpartitioning(c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] - -(35) HashAggregate [codegen id : 1] -Input [9]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39, sum#47] -Keys [8]: [c_customer_id#34, c_first_name#35, c_last_name#36, d_year#46, c_preferred_cust_flag#37, c_birth_country#31, c_login#38, c_email_address#39] -Functions [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23] -Results [5]: [c_customer_id#34 AS customer_id#48, c_first_name#35 AS customer_first_name#49, c_last_name#36 AS customer_last_name#50, c_email_address#39 AS customer_email_address#51, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#42 - ss_ext_discount_amt#41)))#23,18,2) AS year_total#52] - -(36) BroadcastExchange -Input [5]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#48] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] +(27) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] + +(29) CometBroadcastHashJoin +Left output [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42] +Right output [2]: [d_date_sk#44, d_year#45] +Arguments: [ss_sold_date_sk#42], [d_date_sk#44], Inner, BuildRight + +(30) CometProject +Input [12]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, ss_sold_date_sk#42, d_date_sk#44, d_year#45] +Arguments: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45], [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] + +(31) CometHashAggregate +Input [10]: [c_customer_id#33, c_first_name#34, c_last_name#35, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, ss_ext_discount_amt#40, ss_ext_list_price#41, d_year#45] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(32) CometExchange +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [9]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38, sum#46] +Keys [8]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#45, c_preferred_cust_flag#36, c_birth_country#30, c_login#37, c_email_address#38] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#41 - ss_ext_discount_amt#40)))] + +(34) CometBroadcastExchange +Input [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#23, year_total#24] +Right output [5]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Arguments: [customer_id#23], [customer_id#47], Inner, BuildRight + +(36) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(39) CometFilter -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Condition : (isnotnull(c_customer_sk#53) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(37) CometFilter +Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] +Condition : (isnotnull(c_customer_sk#52) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)))) -(40) CometProject -Input [8]: [c_customer_sk#53, c_customer_id#54, c_first_name#55, c_last_name#56, c_preferred_cust_flag#57, c_birth_country#58, c_login#59, c_email_address#60] -Arguments: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#57, 1)) AS c_preferred_cust_flag#64, c_birth_country#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#59, 13)) AS c_login#65, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#60, 50)) AS c_email_address#66] +(38) CometProject +Input [8]: [c_customer_sk#52, c_customer_id#53, c_first_name#54, c_last_name#55, c_preferred_cust_flag#56, c_birth_country#57, c_login#58, c_email_address#59] +Arguments: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65], [c_customer_sk#52, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#53, 16)) AS c_customer_id#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#54, 20)) AS c_first_name#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#55, 30)) AS c_last_name#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#56, 1)) AS c_preferred_cust_flag#63, c_birth_country#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#58, 13)) AS c_login#64, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#59, 50)) AS c_email_address#65] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#70), dynamicpruningexpression(ws_sold_date_sk#70 IN dynamicpruning#71)] +PartitionFilters: [isnotnull(ws_sold_date_sk#69), dynamicpruningexpression(ws_sold_date_sk#69 IN dynamicpruning#70)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Condition : isnotnull(ws_bill_customer_sk#67) +(40) CometFilter +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Condition : isnotnull(ws_bill_customer_sk#66) + +(41) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(43) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(42) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65] +Right output [4]: [ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_sk#52], [ws_bill_customer_sk#66], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66] -Right output [4]: [ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#67], Inner, BuildRight +(43) CometProject +Input [12]: [c_customer_sk#52, c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_bill_customer_sk#66, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] -(45) CometProject -Input [12]: [c_customer_sk#53, c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_bill_customer_sk#67, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#71, d_year#72] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#72, d_year#73] +(45) CometBroadcastHashJoin +Left output [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69] +Right output [2]: [d_date_sk#71, d_year#72] +Arguments: [ws_sold_date_sk#69], [d_date_sk#71], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70] -Right output [2]: [d_date_sk#72, d_year#73] -Arguments: [ws_sold_date_sk#70], [d_date_sk#72], Inner, BuildRight +(46) CometProject +Input [12]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, ws_sold_date_sk#69, d_date_sk#71, d_year#72] +Arguments: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72], [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] -(48) CometProject -Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, ws_sold_date_sk#70, d_date_sk#72, d_year#73] -Arguments: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73], [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] +(47) CometHashAggregate +Input [10]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, ws_ext_discount_amt#67, ws_ext_list_price#68, d_year#72] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] + +(48) CometExchange +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] +Arguments: hashpartitioning(c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, ws_ext_discount_amt#68, ws_ext_list_price#69, d_year#73] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] - -(50) CometExchange -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] -Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] - -(52) HashAggregate [codegen id : 2] -Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73, sum#74] -Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#58, c_login#65, c_email_address#66, d_year#73] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75] -Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#69 - ws_ext_discount_amt#68)))#75,18,2) AS year_total#77] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#76, year_total#77] -Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#76, year_total#77] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#76] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [8]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, year_total#77] -Input [9]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, customer_id#76, year_total#77] - -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Input [9]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72, sum#73] +Keys [8]: [c_customer_id#60, c_first_name#61, c_last_name#62, c_preferred_cust_flag#63, c_birth_country#57, c_login#64, c_email_address#65, d_year#72] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#68 - ws_ext_discount_amt#67)))] + +(50) CometFilter +Input [2]: [customer_id#74, year_total#75] +Condition : (isnotnull(year_total#75) AND (year_total#75 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#74, year_total#75] + +(52) CometBroadcastHashJoin +Left output [7]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51] +Right output [2]: [customer_id#74, year_total#75] +Arguments: [customer_id#23], [customer_id#74], Inner, BuildRight + +(53) CometProject +Input [9]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, customer_id#74, year_total#75] +Arguments: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75], [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) CometFilter -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Condition : (isnotnull(c_customer_sk#78) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)))) +(55) CometFilter +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Condition : (isnotnull(c_customer_sk#76) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)))) -(59) CometProject -Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] -Arguments: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91], [c_customer_sk#78, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#79, 16)) AS c_customer_id#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#80, 20)) AS c_first_name#87, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#81, 30)) AS c_last_name#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#82, 1)) AS c_preferred_cust_flag#89, c_birth_country#83, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#84, 13)) AS c_login#90, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#85, 50)) AS c_email_address#91] +(56) CometProject +Input [8]: [c_customer_sk#76, c_customer_id#77, c_first_name#78, c_last_name#79, c_preferred_cust_flag#80, c_birth_country#81, c_login#82, c_email_address#83] +Arguments: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89], [c_customer_sk#76, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#77, 16)) AS c_customer_id#84, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#78, 20)) AS c_first_name#85, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#79, 30)) AS c_last_name#86, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_preferred_cust_flag#80, 1)) AS c_preferred_cust_flag#87, c_birth_country#81, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_login#82, 13)) AS c_login#88, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_email_address#83, 50)) AS c_email_address#89] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#95), dynamicpruningexpression(ws_sold_date_sk#95 IN dynamicpruning#96)] +PartitionFilters: [isnotnull(ws_sold_date_sk#93), dynamicpruningexpression(ws_sold_date_sk#93 IN dynamicpruning#94)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Condition : isnotnull(ws_bill_customer_sk#92) +(58) CometFilter +Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Condition : isnotnull(ws_bill_customer_sk#90) + +(59) CometBroadcastExchange +Input [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] + +(60) CometBroadcastHashJoin +Left output [8]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89] +Right output [4]: [ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [c_customer_sk#76], [ws_bill_customer_sk#90], Inner, BuildRight -(62) CometBroadcastExchange -Input [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] +(61) CometProject +Input [12]: [c_customer_sk#76, c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_bill_customer_sk#90, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#95, d_year#96] (63) CometBroadcastHashJoin -Left output [8]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91] -Right output [4]: [ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [c_customer_sk#78], [ws_bill_customer_sk#92], Inner, BuildRight +Left output [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93] +Right output [2]: [d_date_sk#95, d_year#96] +Arguments: [ws_sold_date_sk#93], [d_date_sk#95], Inner, BuildRight (64) CometProject -Input [12]: [c_customer_sk#78, c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_bill_customer_sk#92, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Arguments: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95], [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#97, d_year#98] - -(66) CometBroadcastHashJoin -Left output [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95] -Right output [2]: [d_date_sk#97, d_year#98] -Arguments: [ws_sold_date_sk#95], [d_date_sk#97], Inner, BuildRight - -(67) CometProject -Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, ws_sold_date_sk#95, d_date_sk#97, d_year#98] -Arguments: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98], [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98] - -(68) CometHashAggregate -Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, ws_ext_discount_amt#93, ws_ext_list_price#94, d_year#98] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98] -Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] - -(69) CometExchange -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] -Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] - -(71) HashAggregate [codegen id : 3] -Input [9]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98, sum#99] -Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#83, c_login#90, c_email_address#91, d_year#98] -Functions [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))] -Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75] -Results [2]: [c_customer_id#86 AS customer_id#100, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#94 - ws_ext_discount_amt#93)))#75,18,2) AS year_total#101] - -(72) BroadcastExchange -Input [2]: [customer_id#100, year_total#101] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#24] -Right keys [1]: [customer_id#100] -Join type: Inner -Join condition: (CASE WHEN (year_total#77 > 0.00) THEN (year_total#101 / year_total#77) ELSE 0E-20 END > CASE WHEN (year_total#25 > 0.00) THEN (year_total#52 / year_total#25) ELSE 0E-20 END) - -(74) Project [codegen id : 4] -Output [4]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] -Input [10]: [customer_id#24, year_total#25, customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51, year_total#52, year_total#77, customer_id#100, year_total#101] - -(75) TakeOrderedAndProject -Input [4]: [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] -Arguments: 100, [customer_id#48 ASC NULLS FIRST, customer_first_name#49 ASC NULLS FIRST, customer_last_name#50 ASC NULLS FIRST, customer_email_address#51 ASC NULLS FIRST], [customer_id#48, customer_first_name#49, customer_last_name#50, customer_email_address#51] +Input [12]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, ws_sold_date_sk#93, d_date_sk#95, d_year#96] +Arguments: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96], [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] + +(65) CometHashAggregate +Input [10]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, ws_ext_discount_amt#91, ws_ext_list_price#92, d_year#96] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] + +(66) CometExchange +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] +Arguments: hashpartitioning(c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [9]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96, sum#97] +Keys [8]: [c_customer_id#84, c_first_name#85, c_last_name#86, c_preferred_cust_flag#87, c_birth_country#81, c_login#88, c_email_address#89, d_year#96] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#92 - ws_ext_discount_amt#91)))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#98, year_total#99] +Arguments: [customer_id#98, year_total#99] + +(69) CometBroadcastHashJoin +Left output [8]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75] +Right output [2]: [customer_id#98, year_total#99] +Arguments: [customer_id#23], [customer_id#98], Inner, (CASE WHEN (year_total#75 > 0.00) THEN (year_total#99 / year_total#75) ELSE 0E-20 END > CASE WHEN (year_total#24 > 0.00) THEN (year_total#51 / year_total#24) ELSE 0E-20 END), BuildRight + +(70) CometProject +Input [10]: [customer_id#23, year_total#24, customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50, year_total#51, year_total#75, customer_id#98, year_total#99] +Arguments: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] + +(71) CometTakeOrderedAndProject +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[customer_id#47 ASC NULLS FIRST,customer_first_name#48 ASC NULLS FIRST,customer_last_name#49 ASC NULLS FIRST,customer_email_address#50 ASC NULLS FIRST], output=[customer_id#47,customer_first_name#48,customer_last_name#49,customer_email_address#50]), [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50], 100, 0, [customer_id#47 ASC NULLS FIRST, customer_first_name#48 ASC NULLS FIRST, customer_last_name#49 ASC NULLS FIRST, customer_email_address#50 ASC NULLS FIRST], [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] + +(72) CometColumnarToRow [codegen id : 1] +Input [4]: [customer_id#47, customer_first_name#48, customer_last_name#49, customer_email_address#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#18 IN dynamicpruning#19 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#20, d_year#21] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#43 IN dynamicpruning#44 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#42 IN dynamicpruning#43 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#45, d_year#46] +(77) 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_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] ReadSchema: struct -(81) CometFilter -Input [2]: [d_date_sk#45, d_year#46] -Condition : ((isnotnull(d_year#46) AND (d_year#46 = 2002)) AND isnotnull(d_date_sk#45)) +(78) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : ((isnotnull(d_year#45) AND (d_year#45 = 2002)) AND isnotnull(d_date_sk#44)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#45, d_year#46] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#44, d_year#45] -(83) BroadcastExchange -Input [2]: [d_date_sk#45, d_year#46] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#70 IN dynamicpruning#19 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#69 IN dynamicpruning#19 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#95 IN dynamicpruning#44 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#93 IN dynamicpruning#43 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt index 5f4f9a5e17..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#4)))#5,18,2) AS year_total#6)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#7 AS customer_id#8, c_first_name#9 AS customer_first_name#10, c_last_name#11 AS customer_last_name#12, c_email_address#13 AS customer_email_address#14, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#15 - ss_ext_discount_amt#16)))#5,18,2) AS year_total#17)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#18 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#20 - ws_ext_discount_amt#21)))#22,18,2) AS year_total#23)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#24 AS customer_id#25, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#26 - ws_ext_discount_amt#27)))#22,18,2) AS year_total#28)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt index 397a5f6da9..157d1d587e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt index 5737af1936..affae68344 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) TakeOrderedAndProject -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: 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#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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt index 9fdfb28398..3f41c97ff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt index 95364332b9..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + 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 + 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-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index 5737af1936..affae68344 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) TakeOrderedAndProject -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: 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#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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt index 9fdfb28398..3f41c97ff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 95364332b9..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + 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 + 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-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt index 78aa0a8ea8..2e7abc07bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) TakeOrderedAndProject -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: 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#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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt index 93f2c17a1b..cd52b2cd12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt index 1b6a5f13f3..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + 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 + 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-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index 78aa0a8ea8..2e7abc07bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -1,28 +1,27 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +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 @@ -102,68 +101,63 @@ Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) TakeOrderedAndProject -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: 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#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] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (28) ++- * CometColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (24) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(26) CometFilter +(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) CometProject +(26) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(27) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(28) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt index 93f2c17a1b..cd52b2cd12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt @@ -3,30 +3,29 @@ TakeOrderedAndProject +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index 1b6a5f13f3..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -1,38 +1,35 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - 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 - 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 - 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] + 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 + 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-spark4_0/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt index b8a2a2bb8d..7233391f5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt @@ -1,52 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (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) - :- * 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 @@ -154,154 +149,126 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale 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 : 1] -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] - -(24) HashAggregate [codegen 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))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#18, sum(UnscaledValue(ss_ext_sales_price#3))#19] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#18,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#19,17,2)) as decimal(38,11)) AS gross_margin#20, i_category#13 AS i_category#21, i_class#12 AS i_class#22, 0 AS t_category#23, 0 AS t_class#24, 0 AS lochierarchy#25] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#26, i_class#27, sum#28, sum#29] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] - -(27) HashAggregate [codegen id : 2] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] -Keys [2]: [i_category#26, i_class#27] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#30))#32, sum(UnscaledValue(ss_ext_sales_price#31))#33] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#30))#32,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#33,17,2) AS ss_ext_sales_price#35, i_category#26] - -(28) HashAggregate [codegen id : 2] -Input [3]: [ss_net_profit#34, ss_ext_sales_price#35, i_category#26] -Keys [1]: [i_category#26] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] -Results [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] - -(29) CometColumnarExchange -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_category#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 3] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] - -(31) HashAggregate [codegen id : 3] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Keys [1]: [i_category#26] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#44, sum(ss_ext_sales_price#35)#45] -Results [6]: [(sum(ss_net_profit#34)#44 / sum(ss_ext_sales_price#35)#45) AS gross_margin#46, i_category#26, null AS i_class#47, 0 AS t_category#48, 1 AS t_class#49, 1 AS lochierarchy#50] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#51, i_class#52, sum#53, sum#54] - -(33) CometColumnarToRow [codegen id : 4] -Input [4]: [i_category#51, i_class#52, sum#53, sum#54] - -(34) HashAggregate [codegen id : 4] -Input [4]: [i_category#51, i_class#52, sum#53, sum#54] -Keys [2]: [i_category#51, i_class#52] -Functions [2]: [sum(UnscaledValue(ss_net_profit#55)), sum(UnscaledValue(ss_ext_sales_price#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#55))#32, sum(UnscaledValue(ss_ext_sales_price#56))#33] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#55))#32,17,2) AS ss_net_profit#57, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#56))#33,17,2) AS ss_ext_sales_price#58] - -(35) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#57, ss_ext_sales_price#58] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#57), partial_sum(ss_ext_sales_price#58)] -Aggregate Attributes [4]: [sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] -(36) CometColumnarExchange -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#18, i_class#19, sum#20, sum#21] + +(25) CometHashAggregate +Input [4]: [i_category#18, i_class#19, sum#20, sum#21] +Keys [2]: [i_category#18, i_class#19] +Functions [2]: [sum(UnscaledValue(ss_net_profit#22)), sum(UnscaledValue(ss_ext_sales_price#23))] + +(26) CometHashAggregate +Input [3]: [ss_net_profit#24, ss_ext_sales_price#25, i_category#18] +Keys [1]: [i_category#18] +Functions [2]: [partial_sum(ss_net_profit#24), partial_sum(ss_ext_sales_price#25)] + +(27) CometExchange +Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] +Keys [1]: [i_category#18] +Functions [2]: [sum(ss_net_profit#24), sum(ss_ext_sales_price#25)] + +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#30, i_class#31, sum#32, sum#33] + +(30) CometHashAggregate +Input [4]: [i_category#30, i_class#31, sum#32, sum#33] +Keys [2]: [i_category#30, i_class#31] +Functions [2]: [sum(UnscaledValue(ss_net_profit#34)), sum(UnscaledValue(ss_ext_sales_price#35))] + +(31) CometHashAggregate +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Keys: [] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -(37) CometColumnarToRow [codegen id : 5] -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] +(32) CometExchange +Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) HashAggregate [codegen id : 5] -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] +(33) CometHashAggregate +Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] Keys: [] -Functions [2]: [sum(ss_net_profit#57), sum(ss_ext_sales_price#58)] -Aggregate Attributes [2]: [sum(ss_net_profit#57)#67, sum(ss_ext_sales_price#58)#68] -Results [6]: [(sum(ss_net_profit#57)#67 / sum(ss_ext_sales_price#58)#68) AS gross_margin#69, null AS i_category#70, null AS i_class#71, 1 AS t_category#72, 1 AS t_class#73, 2 AS lochierarchy#74] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -(39) Union +(34) CometUnion +Child 0 Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Child 1 Input [6]: [gross_margin#48, i_category#18, i_class#49, t_category#50, t_class#51, lochierarchy#52] +Child 2 Input [6]: [gross_margin#53, i_category#54, i_class#55, t_category#56, t_class#57, lochierarchy#58] -(40) HashAggregate [codegen id : 6] -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Keys [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] +(35) CometHashAggregate +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -(41) CometColumnarExchange -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Arguments: hashpartitioning(gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(36) CometExchange +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Arguments: hashpartitioning(gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometHashAggregate -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Keys [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] +(37) CometHashAggregate +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] Functions: [] -(43) CometExchange -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(38) CometExchange +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: hashpartitioning(lochierarchy#47, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(44) CometSort -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] +(39) CometSort +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59], [lochierarchy#47 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, gross_margin#42 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 7] -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +(40) CometColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -(46) Window -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] +(41) Window +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: [rank(gross_margin#42) windowspecdefinition(lochierarchy#47, _w0#59, gross_margin#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#47, _w0#59], [gross_margin#42 ASC NULLS FIRST] -(47) Project [codegen id : 8] -Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] +Input [6]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59, rank_within_parent#60] -(48) TakeOrderedAndProject -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(43) TakeOrderedAndProject +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] +Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0) THEN i_category#43 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(50) CometFilter +(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)) -(51) CometProject +(46) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(52) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(53) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#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-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt index 9e1bf58e71..1f6c984b22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt @@ -5,106 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: Unsupported result expressions found in: List(cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) as decimal(38,11)) AS gross_margin#5, i_category#6 AS i_category#7, i_class#8 AS i_class#9, 0 AS t_category#10, 0 AS t_class#11, 0 AS lochierarchy#12)] - : +- CometColumnarToRow - : +- CometExchange + +- 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 - : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_net_profit#13))#14,17,2) AS ss_net_profit#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#16))#17,17,2) AS ss_ext_sales_price#18, i_category#19)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_net_profit#20))#14,17,2) AS ss_net_profit#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#22))#17,17,2) AS ss_ext_sales_price#23)] - +- CometColumnarToRow - +- 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 + : +- 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 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt index b799504154..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt @@ -1,72 +1,55 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (7) + WholeStageCodegen (1) 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 (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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 - 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] - WholeStageCodegen (3) - 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] #8 - WholeStageCodegen (2) - 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 (5) - 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 #9 - WholeStageCodegen (4) - 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 - ReusedExchange [i_category,i_class,sum,sum] #3 + 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 + 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-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt index b8a2a2bb8d..7233391f5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -1,52 +1,47 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometExchange (43) - +- CometHashAggregate (42) - +- CometColumnarExchange (41) - +- * HashAggregate (40) - +- Union (39) - :- * HashAggregate (24) - : +- * CometColumnarToRow (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) - :- * 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 @@ -154,154 +149,126 @@ Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(Unscale 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 : 1] -Input [4]: [i_category#13, i_class#12, sum#16, sum#17] - -(24) HashAggregate [codegen 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))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#18, sum(UnscaledValue(ss_ext_sales_price#3))#19] -Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#18,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#19,17,2)) as decimal(38,11)) AS gross_margin#20, i_category#13 AS i_category#21, i_class#12 AS i_class#22, 0 AS t_category#23, 0 AS t_class#24, 0 AS lochierarchy#25] - -(25) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#26, i_class#27, sum#28, sum#29] - -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] - -(27) HashAggregate [codegen id : 2] -Input [4]: [i_category#26, i_class#27, sum#28, sum#29] -Keys [2]: [i_category#26, i_class#27] -Functions [2]: [sum(UnscaledValue(ss_net_profit#30)), sum(UnscaledValue(ss_ext_sales_price#31))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#30))#32, sum(UnscaledValue(ss_ext_sales_price#31))#33] -Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#30))#32,17,2) AS ss_net_profit#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#31))#33,17,2) AS ss_ext_sales_price#35, i_category#26] - -(28) HashAggregate [codegen id : 2] -Input [3]: [ss_net_profit#34, ss_ext_sales_price#35, i_category#26] -Keys [1]: [i_category#26] -Functions [2]: [partial_sum(ss_net_profit#34), partial_sum(ss_ext_sales_price#35)] -Aggregate Attributes [4]: [sum#36, isEmpty#37, sum#38, isEmpty#39] -Results [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] - -(29) CometColumnarExchange -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Arguments: hashpartitioning(i_category#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] - -(30) CometColumnarToRow [codegen id : 3] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] - -(31) HashAggregate [codegen id : 3] -Input [5]: [i_category#26, sum#40, isEmpty#41, sum#42, isEmpty#43] -Keys [1]: [i_category#26] -Functions [2]: [sum(ss_net_profit#34), sum(ss_ext_sales_price#35)] -Aggregate Attributes [2]: [sum(ss_net_profit#34)#44, sum(ss_ext_sales_price#35)#45] -Results [6]: [(sum(ss_net_profit#34)#44 / sum(ss_ext_sales_price#35)#45) AS gross_margin#46, i_category#26, null AS i_class#47, 0 AS t_category#48, 1 AS t_class#49, 1 AS lochierarchy#50] - -(32) ReusedExchange [Reuses operator id: 22] -Output [4]: [i_category#51, i_class#52, sum#53, sum#54] - -(33) CometColumnarToRow [codegen id : 4] -Input [4]: [i_category#51, i_class#52, sum#53, sum#54] - -(34) HashAggregate [codegen id : 4] -Input [4]: [i_category#51, i_class#52, sum#53, sum#54] -Keys [2]: [i_category#51, i_class#52] -Functions [2]: [sum(UnscaledValue(ss_net_profit#55)), sum(UnscaledValue(ss_ext_sales_price#56))] -Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#55))#32, sum(UnscaledValue(ss_ext_sales_price#56))#33] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#55))#32,17,2) AS ss_net_profit#57, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#56))#33,17,2) AS ss_ext_sales_price#58] - -(35) HashAggregate [codegen id : 4] -Input [2]: [ss_net_profit#57, ss_ext_sales_price#58] -Keys: [] -Functions [2]: [partial_sum(ss_net_profit#57), partial_sum(ss_ext_sales_price#58)] -Aggregate Attributes [4]: [sum#59, isEmpty#60, sum#61, isEmpty#62] -Results [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] -(36) CometColumnarExchange -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(24) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#18, i_class#19, sum#20, sum#21] + +(25) CometHashAggregate +Input [4]: [i_category#18, i_class#19, sum#20, sum#21] +Keys [2]: [i_category#18, i_class#19] +Functions [2]: [sum(UnscaledValue(ss_net_profit#22)), sum(UnscaledValue(ss_ext_sales_price#23))] + +(26) CometHashAggregate +Input [3]: [ss_net_profit#24, ss_ext_sales_price#25, i_category#18] +Keys [1]: [i_category#18] +Functions [2]: [partial_sum(ss_net_profit#24), partial_sum(ss_ext_sales_price#25)] + +(27) CometExchange +Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(28) CometHashAggregate +Input [5]: [i_category#18, sum#26, isEmpty#27, sum#28, isEmpty#29] +Keys [1]: [i_category#18] +Functions [2]: [sum(ss_net_profit#24), sum(ss_ext_sales_price#25)] + +(29) ReusedExchange [Reuses operator id: 22] +Output [4]: [i_category#30, i_class#31, sum#32, sum#33] + +(30) CometHashAggregate +Input [4]: [i_category#30, i_class#31, sum#32, sum#33] +Keys [2]: [i_category#30, i_class#31] +Functions [2]: [sum(UnscaledValue(ss_net_profit#34)), sum(UnscaledValue(ss_ext_sales_price#35))] + +(31) CometHashAggregate +Input [2]: [ss_net_profit#36, ss_ext_sales_price#37] +Keys: [] +Functions [2]: [partial_sum(ss_net_profit#36), partial_sum(ss_ext_sales_price#37)] -(37) CometColumnarToRow [codegen id : 5] -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] +(32) CometExchange +Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) HashAggregate [codegen id : 5] -Input [4]: [sum#63, isEmpty#64, sum#65, isEmpty#66] +(33) CometHashAggregate +Input [4]: [sum#38, isEmpty#39, sum#40, isEmpty#41] Keys: [] -Functions [2]: [sum(ss_net_profit#57), sum(ss_ext_sales_price#58)] -Aggregate Attributes [2]: [sum(ss_net_profit#57)#67, sum(ss_ext_sales_price#58)#68] -Results [6]: [(sum(ss_net_profit#57)#67 / sum(ss_ext_sales_price#58)#68) AS gross_margin#69, null AS i_category#70, null AS i_class#71, 1 AS t_category#72, 1 AS t_class#73, 2 AS lochierarchy#74] +Functions [2]: [sum(ss_net_profit#36), sum(ss_ext_sales_price#37)] -(39) Union +(34) CometUnion +Child 0 Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Child 1 Input [6]: [gross_margin#48, i_category#18, i_class#49, t_category#50, t_class#51, lochierarchy#52] +Child 2 Input [6]: [gross_margin#53, i_category#54, i_class#55, t_category#56, t_class#57, lochierarchy#58] -(40) HashAggregate [codegen id : 6] -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Keys [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] +(35) CometHashAggregate +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] Functions: [] -Aggregate Attributes: [] -Results [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -(41) CometColumnarExchange -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Arguments: hashpartitioning(gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(36) CometExchange +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Arguments: hashpartitioning(gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometHashAggregate -Input [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] -Keys [6]: [gross_margin#20, i_category#21, i_class#22, t_category#23, t_class#24, lochierarchy#25] +(37) CometHashAggregate +Input [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] +Keys [6]: [gross_margin#42, i_category#43, i_class#44, t_category#45, t_class#46, lochierarchy#47] Functions: [] -(43) CometExchange -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(38) CometExchange +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: hashpartitioning(lochierarchy#47, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(44) CometSort -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] +(39) CometSort +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59], [lochierarchy#47 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, gross_margin#42 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 7] -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +(40) CometColumnarToRow [codegen id : 1] +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] -(46) Window -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] +(41) Window +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59] +Arguments: [rank(gross_margin#42) windowspecdefinition(lochierarchy#47, _w0#59, gross_margin#42 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#47, _w0#59], [gross_margin#42 ASC NULLS FIRST] -(47) Project [codegen id : 8] -Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] +Input [6]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, _w0#59, rank_within_parent#60] -(48) TakeOrderedAndProject -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(43) TakeOrderedAndProject +Input [5]: [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] +Arguments: 100, [lochierarchy#47 DESC NULLS LAST, CASE WHEN (lochierarchy#47 = 0) THEN i_category#43 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [gross_margin#42, i_category#43, i_class#44, lochierarchy#47, rank_within_parent#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(50) CometFilter +(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)) -(51) CometProject +(46) CometProject Input [2]: [d_date_sk#7, d_year#8] Arguments: [d_date_sk#7], [d_date_sk#7] -(52) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#7] -(53) BroadcastExchange +(48) BroadcastExchange Input [1]: [d_date_sk#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-spark4_0/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt index 9e1bf58e71..1f6c984b22 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt @@ -5,106 +5,101 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: Unsupported result expressions found in: List(cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) as decimal(38,11)) AS gross_margin#5, i_category#6 AS i_category#7, i_class#8 AS i_class#9, 0 AS t_category#10, 0 AS t_class#11, 0 AS lochierarchy#12)] - : +- CometColumnarToRow - : +- CometExchange + +- 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 - : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_net_profit#13))#14,17,2) AS ss_net_profit#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#16))#17,17,2) AS ss_ext_sales_price#18, i_category#19)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ss_net_profit#20))#14,17,2) AS ss_net_profit#21, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#22))#17,17,2) AS ss_ext_sales_price#23)] - +- CometColumnarToRow - +- 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 + : +- 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 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index b799504154..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -1,72 +1,55 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (7) + WholeStageCodegen (1) 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 (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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 - 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] - WholeStageCodegen (3) - 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] #8 - WholeStageCodegen (2) - 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 (5) - 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 #9 - WholeStageCodegen (4) - 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 - ReusedExchange [i_category,i_class,sum,sum] #3 + 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 + 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-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt index b72fd62948..bf8cb9a216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] - -(22) HashAggregate [codegen id : 1] +(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))#18] -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))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#18,17,2) AS _w0#20] -(23) 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#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) 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#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(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) 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#19, _w0#20] -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#21], [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] -(27) Filter [codegen id : 3] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(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)) -(28) 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#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) 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#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(29) Filter [codegen id : 10] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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 [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] -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) HashAggregate [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] -Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] +(31) CometHashAggregate +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -(34) CometColumnarExchange -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 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] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -(37) Window -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, 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#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +(36) Project [codegen id : 4] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] -(39) BroadcastExchange -Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(37) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 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#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, 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#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(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#30] +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#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 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] -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, 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#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(44) Project [codegen id : 6] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(45) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 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#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(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#30 AS psum#40, sum_sales#38 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#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -(50) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(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 = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt index 7c578b0474..b50b570b4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt index b08d56b5ca..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (10) + 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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - 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 - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (3) 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] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt index b72fd62948..bf8cb9a216 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum#17] - -(22) HashAggregate [codegen id : 1] +(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))#18] -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))#18,17,2) AS sum_sales#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#8))#18,17,2) AS _w0#20] -(23) 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#19, _w0#20] -Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) 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#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +(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) 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#19, _w0#20] -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#21], [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] -(27) Filter [codegen id : 3] -Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] +(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)) -(28) 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#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) 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#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] - -(29) Filter [codegen id : 10] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, 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 [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] -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] +(30) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] -(33) HashAggregate [codegen id : 4] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] -Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] +(31) CometHashAggregate +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] -(34) CometColumnarExchange -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(33) CometSort +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30], [i_category#22 ASC NULLS FIRST, i_brand#23 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] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] -(37) Window -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +(35) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, 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#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +(36) Project [codegen id : 4] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] -(39) BroadcastExchange -Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(37) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] 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] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 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#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, 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#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(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#30] +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#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(41) CometSort +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38], [i_category#32 ASC NULLS FIRST, i_brand#33 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] -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] +(43) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, 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#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(44) Project [codegen id : 6] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(45) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] 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) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 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#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(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#30 AS psum#40, sum_sales#38 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#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] -(50) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(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 = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt index 7c578b0474..b50b570b4b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index b08d56b5ca..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (10) + 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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum, Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - 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 - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) + WholeStageCodegen (3) 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] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt index d850c164a2..2cd07a9fb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt @@ -1,82 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * Filter (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- Window (66) - : +- * CometColumnarToRow (65) - : +- CometSort (64) - : +- CometExchange (63) - : +- CometProject (62) - : +- CometFilter (61) - : +- CometSortMergeJoin (60) - : :- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (17) - : : : +- Window (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (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 (23) - : : +- * Project (22) - : : +- Window (21) - : : +- * CometColumnarToRow (20) - : : +- CometSort (19) - : : +- ReusedExchange (18) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- * HashAggregate (57) - : +- * CometColumnarToRow (56) - : +- CometColumnarExchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (45) - : : +- Window (44) - : : +- * CometColumnarToRow (43) - : : +- CometSort (42) - : : +- CometColumnarExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (33) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : +- ReusedExchange (34) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- Window (49) - : +- * CometColumnarToRow (48) - : +- CometSort (47) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * Project (71) - +- Window (70) - +- * CometColumnarToRow (69) - +- CometSort (68) - +- ReusedExchange (67) +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 @@ -128,333 +126,323 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(12) HashAggregate [codegen id : 1] +(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] -(13) 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] -(14) 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] -(15) CometColumnarToRow [codegen id : 2] -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] -(16) 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] -(17) Project [codegen id : 6] -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] -(18) ReusedExchange [Reuses operator id: 13] -Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -(19) CometSort -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] +(18) CometSort +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] +Arguments: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15], [ws_item_sk#15 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -(21) Window -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +(20) Window +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] +Arguments: [row_number() windowspecdefinition(ws_item_sk#15, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#16], [ws_item_sk#15], [d_date#13 ASC NULLS FIRST] -(22) Project [codegen id : 5] -Output [3]: [item_sk#13, sumws#15, rk#17] -Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] +(21) Project [codegen id : 3] +Output [3]: [item_sk#12, sumws#14, rk#16] +Input [5]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15, rk#16] -(23) BroadcastExchange -Input [3]: [item_sk#13, sumws#15, rk#17] +(22) BroadcastExchange +Input [3]: [item_sk#12, sumws#14, rk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#12] Join type: Inner -Join condition: (rk#12 >= rk#17) - -(25) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] - -(26) HashAggregate [codegen id : 6] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) 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] - -(28) CometColumnarToRow [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(29) HashAggregate [codegen id : 7] -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#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] - -(30) 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] - -(31) 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] - -(32) 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#16) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#12, sumws#14, rk#16] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#14)] +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#14)] +Aggregate Attributes [1]: [sum(sumws#14)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#14)#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 -(33) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(34) ReusedExchange [Reuses operator id: 6] -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) -(35) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(36) CometProject -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, 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 -(37) CometHashAggregate -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))] +(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] -(38) CometExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [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))] -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] +(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] -(40) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -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))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, 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))] -(41) CometColumnarExchange -Input [4]: [item_sk#32, d_date#29, sumss#33, 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] -(42) CometSort -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#29, sumss#33, 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] -(43) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(44) Window -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [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] -(45) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] +(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] -(46) ReusedExchange [Reuses operator id: 41] -Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -(47) CometSort -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(45) CometSort +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] +Arguments: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST] -(48) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -(49) Window -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] +(47) Window +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] +Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#34 ASC NULLS FIRST] -(50) Project [codegen id : 12] -Output [3]: [item_sk#35, sumss#37, rk#39] -Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] +(48) Project [codegen id : 8] +Output [3]: [item_sk#33, sumss#35, rk#37] +Input [5]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36, rk#37] -(51) BroadcastExchange -Input [3]: [item_sk#35, sumss#37, rk#39] +(49) BroadcastExchange +Input [3]: [item_sk#33, sumss#35, rk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#35] +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#33] Join type: Inner -Join condition: (rk#34 >= rk#39) +Join condition: (rk#32 >= rk#37) -(53) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#33, sumss#35, rk#37] -(54) HashAggregate [codegen id : 13] -Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#35)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(55) CometColumnarExchange -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 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] -(56) CometColumnarToRow [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(57) HashAggregate [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#44] -Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] +(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#35)] +Aggregate Attributes [1]: [sum(sumss#35)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#35)#42 AS cume_sales#43] -(58) CometColumnarExchange -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, 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] -(59) CometSort -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 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] -(60) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, 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 -(61) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 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) -(62) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] +(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] -(63) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 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] -(64) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 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] -(65) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(66) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 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] -(67) ReusedExchange [Reuses operator id: 63] -Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -(68) CometSort -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] +(66) CometSort +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] +Arguments: [item_sk#49, d_date#50, web_sales#51, store_sales#52], [item_sk#49 ASC NULLS FIRST, d_date#50 ASC NULLS FIRST] -(69) CometColumnarToRow [codegen id : 30] -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -(70) Window -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] +(68) Window +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] +Arguments: [row_number() windowspecdefinition(item_sk#49, d_date#50 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#53], [item_sk#49], [d_date#50 ASC NULLS FIRST] -(71) Project [codegen id : 31] -Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] +(69) Project [codegen id : 23] +Output [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] +Input [5]: [item_sk#49, d_date#50, web_sales#51, store_sales#52, rk#53] -(72) BroadcastExchange -Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +(70) BroadcastExchange +Input [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(73) BroadcastHashJoin [codegen id : 32] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#51] +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#49] Join type: Inner -Join condition: (rk#50 >= rk#55) - -(74) Project [codegen id : 32] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] - -(75) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] - -(76) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] - -(77) Filter [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(78) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Join condition: (rk#48 >= rk#53) + +(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#49, web_sales#51, store_sales#52, rk#53] + +(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#54, max#55] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] +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)#58, max(store_sales#52)#59] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#58 AS web_cumulative#60, max(store_sales#52)#59 AS store_cumulative#61] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] +Condition : ((isnotnull(web_cumulative#60) AND isnotnull(store_cumulative#61)) AND (web_cumulative#60 > store_cumulative#61)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] +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#60, store_cumulative#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(80) CometFilter +(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) CometProject +(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] -(82) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(83) 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 = 32 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-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index bd8f92b6c0..f7d3371108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt @@ -23,49 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- 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 + : : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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 @@ -78,39 +76,37 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -132,49 +128,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- 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 + : +- 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 @@ -187,38 +181,36 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- 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 + +- 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 130 out of 196 eligible operators (66%). Final plan contains 26 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-spark4_0/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt index c88883dbad..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (32) + 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] @@ -7,7 +7,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (15) + WholeStageCodegen (11) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] @@ -17,105 +17,99 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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 (7) + 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 (6) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (5) + WholeStageCodegen (3) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (2) 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] #9 - WholeStageCodegen (14) + 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 (13) + 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 (9) + WholeStageCodegen (6) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - 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 + 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 (12) + WholeStageCodegen (8) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (11) + WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (31) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (30) + WholeStageCodegen (22) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index d850c164a2..2cd07a9fb7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -1,82 +1,80 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * Filter (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- Window (66) - : +- * CometColumnarToRow (65) - : +- CometSort (64) - : +- CometExchange (63) - : +- CometProject (62) - : +- CometFilter (61) - : +- CometSortMergeJoin (60) - : :- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (17) - : : : +- Window (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (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 (23) - : : +- * Project (22) - : : +- Window (21) - : : +- * CometColumnarToRow (20) - : : +- CometSort (19) - : : +- ReusedExchange (18) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- * HashAggregate (57) - : +- * CometColumnarToRow (56) - : +- CometColumnarExchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (45) - : : +- Window (44) - : : +- * CometColumnarToRow (43) - : : +- CometSort (42) - : : +- CometColumnarExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (33) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : +- ReusedExchange (34) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- Window (49) - : +- * CometColumnarToRow (48) - : +- CometSort (47) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * Project (71) - +- Window (70) - +- * CometColumnarToRow (69) - +- CometSort (68) - +- ReusedExchange (67) +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 @@ -128,333 +126,323 @@ Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [3]: [ws_item_sk#1, d_date#6, sum#8] - -(12) HashAggregate [codegen id : 1] +(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] -(13) 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] -(14) 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] -(15) CometColumnarToRow [codegen id : 2] -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] -(16) 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] -(17) Project [codegen id : 6] -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] -(18) ReusedExchange [Reuses operator id: 13] -Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +(17) ReusedExchange [Reuses operator id: 12] +Output [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -(19) CometSort -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] +(18) CometSort +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] +Arguments: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15], [ws_item_sk#15 ASC NULLS FIRST, d_date#13 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] -(21) Window -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +(20) Window +Input [4]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15] +Arguments: [row_number() windowspecdefinition(ws_item_sk#15, d_date#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#16], [ws_item_sk#15], [d_date#13 ASC NULLS FIRST] -(22) Project [codegen id : 5] -Output [3]: [item_sk#13, sumws#15, rk#17] -Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] +(21) Project [codegen id : 3] +Output [3]: [item_sk#12, sumws#14, rk#16] +Input [5]: [item_sk#12, d_date#13, sumws#14, ws_item_sk#15, rk#16] -(23) BroadcastExchange -Input [3]: [item_sk#13, sumws#15, rk#17] +(22) BroadcastExchange +Input [3]: [item_sk#12, sumws#14, rk#16] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#12] Join type: Inner -Join condition: (rk#12 >= rk#17) - -(25) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] - -(26) HashAggregate [codegen id : 6] -Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] -Keys [3]: [item_sk#10, d_date#6, sumws#11] -Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(27) 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] - -(28) CometColumnarToRow [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(29) HashAggregate [codegen id : 7] -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#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] - -(30) 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] - -(31) 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] - -(32) 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#16) + +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] +Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#12, sumws#14, rk#16] + +(25) HashAggregate [codegen id : 4] +Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#14] +Keys [3]: [item_sk#9, d_date#6, sumws#10] +Functions [1]: [partial_sum(sumws#14)] +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#14)] +Aggregate Attributes [1]: [sum(sumws#14)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#14)#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 -(33) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(34) ReusedExchange [Reuses operator id: 6] -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) -(35) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] -(36) CometProject -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, 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 -(37) CometHashAggregate -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))] +(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] -(38) CometExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [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))] -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] +(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] -(40) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -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))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, 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))] -(41) CometColumnarExchange -Input [4]: [item_sk#32, d_date#29, sumss#33, 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] -(42) CometSort -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#29, sumss#33, 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] -(43) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(44) Window -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [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] -(45) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] +(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] -(46) ReusedExchange [Reuses operator id: 41] -Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -(47) CometSort -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(45) CometSort +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] +Arguments: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36], [ss_item_sk#36 ASC NULLS FIRST, d_date#34 ASC NULLS FIRST] -(48) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] -(49) Window -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] +(47) Window +Input [4]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36] +Arguments: [row_number() windowspecdefinition(ss_item_sk#36, d_date#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#37], [ss_item_sk#36], [d_date#34 ASC NULLS FIRST] -(50) Project [codegen id : 12] -Output [3]: [item_sk#35, sumss#37, rk#39] -Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] +(48) Project [codegen id : 8] +Output [3]: [item_sk#33, sumss#35, rk#37] +Input [5]: [item_sk#33, d_date#34, sumss#35, ss_item_sk#36, rk#37] -(51) BroadcastExchange -Input [3]: [item_sk#35, sumss#37, rk#39] +(49) BroadcastExchange +Input [3]: [item_sk#33, sumss#35, rk#37] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#35] +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#33] Join type: Inner -Join condition: (rk#34 >= rk#39) +Join condition: (rk#32 >= rk#37) -(53) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#33, sumss#35, rk#37] -(54) HashAggregate [codegen id : 13] -Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#35] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#35)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(55) CometColumnarExchange -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 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] -(56) CometColumnarToRow [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -(57) HashAggregate [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#44] -Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] +(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#35)] +Aggregate Attributes [1]: [sum(sumss#35)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#35)#42 AS cume_sales#43] -(58) CometColumnarExchange -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, 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] -(59) CometSort -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 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] -(60) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, 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 -(61) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 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) -(62) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] +(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] -(63) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 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] -(64) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 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] -(65) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -(66) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 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] -(67) ReusedExchange [Reuses operator id: 63] -Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -(68) CometSort -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] +(66) CometSort +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] +Arguments: [item_sk#49, d_date#50, web_sales#51, store_sales#52], [item_sk#49 ASC NULLS FIRST, d_date#50 ASC NULLS FIRST] -(69) CometColumnarToRow [codegen id : 30] -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] -(70) Window -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] +(68) Window +Input [4]: [item_sk#49, d_date#50, web_sales#51, store_sales#52] +Arguments: [row_number() windowspecdefinition(item_sk#49, d_date#50 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#53], [item_sk#49], [d_date#50 ASC NULLS FIRST] -(71) Project [codegen id : 31] -Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] +(69) Project [codegen id : 23] +Output [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] +Input [5]: [item_sk#49, d_date#50, web_sales#51, store_sales#52, rk#53] -(72) BroadcastExchange -Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +(70) BroadcastExchange +Input [4]: [item_sk#49, web_sales#51, store_sales#52, rk#53] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] -(73) BroadcastHashJoin [codegen id : 32] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#51] +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#49] Join type: Inner -Join condition: (rk#50 >= rk#55) - -(74) Project [codegen id : 32] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] - -(75) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] - -(76) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] - -(77) Filter [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(78) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Join condition: (rk#48 >= rk#53) + +(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#49, web_sales#51, store_sales#52, rk#53] + +(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#54, max#55] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#56, max#57] +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)#58, max(store_sales#52)#59] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#58 AS web_cumulative#60, max(store_sales#52)#59 AS store_cumulative#61] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] +Condition : ((isnotnull(web_cumulative#60) AND isnotnull(store_cumulative#61)) AND (web_cumulative#60 > store_cumulative#61)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#60, store_cumulative#61] +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#60, store_cumulative#61] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(80) CometFilter +(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) CometProject +(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] -(82) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(83) 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 = 32 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-spark4_0/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt index bd8f92b6c0..f7d3371108 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt @@ -23,49 +23,47 @@ TakeOrderedAndProject : : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : : +- CometColumnarToRow : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- 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 + : : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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 @@ -78,39 +76,37 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- 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 + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] @@ -132,49 +128,47 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- 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 + : +- 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 @@ -187,38 +181,36 @@ TakeOrderedAndProject : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- 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 + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- 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 + +- 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 130 out of 196 eligible operators (66%). Final plan contains 26 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-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index c88883dbad..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (32) + 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] @@ -7,7 +7,7 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store BroadcastHashJoin [item_sk,item_sk,rk,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (15) + WholeStageCodegen (11) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] @@ -17,105 +17,99 @@ TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store 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 (7) + 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 (6) + 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 (2) + WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (5) + WholeStageCodegen (3) Project [item_sk,sumws,rk] InputAdapter Window [ws_item_sk,d_date] - WholeStageCodegen (4) + WholeStageCodegen (2) 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] #9 - WholeStageCodegen (14) + 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 (13) + 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 (9) + WholeStageCodegen (6) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - 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 + 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 (12) + WholeStageCodegen (8) Project [item_sk,sumss,rk] InputAdapter Window [ss_item_sk,d_date] - WholeStageCodegen (11) + WholeStageCodegen (7) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,sumss,ss_item_sk] ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 InputAdapter BroadcastExchange #14 - WholeStageCodegen (31) + WholeStageCodegen (23) Project [item_sk,web_sales,store_sales,rk] InputAdapter Window [item_sk,d_date] - WholeStageCodegen (30) + WholeStageCodegen (22) CometColumnarToRow InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt index e5b34056ab..9265c65475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] - -(22) HashAggregate [codegen id : 1] +(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))#17] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS _w0#19] -(23) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(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) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15, 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, 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] -(27) Filter [codegen id : 3] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(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)) -(28) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(29) Filter [codegen id : 10] -Input [9]: [i_category#5, i_brand#4, cc_name#15, 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) +(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] -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) HashAggregate [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] -Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] -Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] +(31) CometHashAggregate +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -(34) CometColumnarExchange -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -(37) Window -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +(36) Project [codegen id : 4] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] -(39) BroadcastExchange -Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(37) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 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#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(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#28] +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#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(44) Project [codegen id : 6] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(45) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] 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) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 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#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(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#28 AS psum#37, sum_sales#35 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#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -(50) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(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 = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt index ba90418971..de7d26cd06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt index 29a73f88ac..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (10) + 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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index e5b34056ab..9265c65475 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -1,54 +1,52 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (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 (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +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 @@ -148,153 +146,143 @@ Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#8))] 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] -(21) CometColumnarToRow [codegen id : 1] -Input [6]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum#16] - -(22) HashAggregate [codegen id : 1] +(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))#17] -Results [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(cs_sales_price#8))#17,17,2) AS _w0#19] -(23) CometColumnarExchange -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(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) CometSort -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [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] -(25) CometColumnarToRow [codegen id : 2] -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +(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) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, 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, cc_name#15, 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, 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] -(27) Filter [codegen id : 3] -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] +(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)) -(28) Window -Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] - -(29) Filter [codegen id : 10] -Input [9]: [i_category#5, i_brand#4, cc_name#15, 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) +(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] -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +(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) -(31) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(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] -(32) CometColumnarToRow [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] +(30) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] -(33) HashAggregate [codegen id : 4] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] -Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] -Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] -Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] +(31) CometHashAggregate +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] -(34) CometColumnarExchange -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +(32) CometExchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(35) CometSort -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(33) CometSort +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28], [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] -(37) Window -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +(35) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +(36) Project [codegen id : 4] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] -(39) BroadcastExchange -Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(37) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 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#21, i_brand#22, cc_name#23, (rn#29 + 1)] Join type: Inner Join condition: None -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(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#28] +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#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(41) CometSort +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35], [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] +(43) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(44) Project [codegen id : 6] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(45) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] 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) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 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#30, i_brand#31, cc_name#32, (rn#36 - 1)] Join type: Inner Join condition: None -(49) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(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#28 AS psum#37, sum_sales#35 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#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] -(50) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(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 = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(51) 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 -(52) 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)) -(53) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) 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=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt index ba90418971..de7d26cd06 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt @@ -10,99 +10,96 @@ TakeOrderedAndProject : : +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : : +- CometColumnarToRow : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- 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 + : : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] : +- CometColumnarToRow : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- 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 + : +- 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: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - +- CometColumnarToRow - +- 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 + +- 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 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index 29a73f88ac..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -1,5 +1,5 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (10) + 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] @@ -8,69 +8,63 @@ TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_m Filter [avg_monthly_sales,sum_sales] InputAdapter Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) + WholeStageCodegen (2) Filter [d_year] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) + WholeStageCodegen (1) 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 (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - 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] + 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 (6) + WholeStageCodegen (4) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) + WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - 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 + 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 (9) + WholeStageCodegen (6) Project [i_category,i_brand,cc_name,sum_sales,rn] InputAdapter Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) + WholeStageCodegen (5) CometColumnarToRow InputAdapter CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt index a3c14dc037..ca9ea23933 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (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) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* 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 @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(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))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(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) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(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) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(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] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(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] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(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) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(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#62, cp_catalog_page_id#63] +(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#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#63, 16)) AS cp_catalog_page_id#64] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(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] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(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 -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(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] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(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))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(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] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(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))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(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) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(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] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(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 -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(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)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(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] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(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 -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#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] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#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] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(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 -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(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 -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#106, 16)) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(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 -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], 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] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(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]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -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#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [2]: [channel#139, id#140] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#133, sum(returns#148)#134, sum(profit#149)#135] -Results [4]: [channel#139, sum(sales#147)#133 AS sales#150, sum(returns#148)#134 AS returns#151, sum(profit#149)#135 AS profit#152] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#139, sales#150, returns#151, profit#152] -Keys [1]: [channel#139] -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#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(77) CometColumnarExchange -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: hashpartitioning(channel#139, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Keys [1]: [channel#139] -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#139, 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] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] -Keys [2]: [channel#172, id#173] -Functions [3]: [sum(sales#180), sum(returns#181), sum(profit#182)] -Aggregate Attributes [3]: [sum(sales#180)#133, sum(returns#181)#134, sum(profit#182)#135] -Results [3]: [sum(sales#180)#133 AS sales#183, sum(returns#181)#134 AS returns#184, sum(profit#182)#135 AS profit#185] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#183, returns#184, profit#185] +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#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(70) CometHashAggregate +Input [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [2]: [channel#115, id#116] +Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] + +(71) CometHashAggregate +Input [4]: [channel#115, sales#126, returns#127, profit#128] +Keys [1]: [channel#115] +Functions [3]: [partial_sum(sales#126), partial_sum(returns#127), partial_sum(profit#128)] + +(72) CometExchange +Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Arguments: hashpartitioning(channel#115, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [1]: [channel#115] +Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] + +(75) CometHashAggregate +Input [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Keys [2]: [channel#135, id#136] +Functions [3]: [sum(sales#143), sum(returns#144), sum(profit#145)] + +(76) CometHashAggregate +Input [3]: [sales#146, returns#147, profit#148] Keys: [] -Functions [3]: [partial_sum(sales#183), partial_sum(returns#184), partial_sum(profit#185)] -Aggregate Attributes [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -Results [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] - -(84) CometColumnarExchange -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#146), partial_sum(returns#147), partial_sum(profit#148)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +(77) CometExchange +Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +(78) CometHashAggregate +Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] Keys: [] -Functions [3]: [sum(sales#183), sum(returns#184), sum(profit#185)] -Aggregate Attributes [3]: [sum(sales#183)#198, sum(returns#184)#199, sum(profit#185)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#183)#198 AS sum(sales)#203, sum(returns#184)#199 AS sum(returns)#204, sum(profit#185)#200 AS sum(profit)#205] +Functions [3]: [sum(sales#146), sum(returns#147), sum(profit#148)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Child 1 Input [5]: [channel#115, id#158, sum(sales)#159, sum(returns)#160, sum(profit)#161] +Child 2 Input [5]: [channel#162, id#163, sum(sales)#164, sum(returns)#165, sum(profit)#166] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Arguments: hashpartitioning(channel#94, id#95, sales#155, returns#156, profit#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#155,returns#156,profit#157]), [channel#94, id#95, sales#155, returns#156, profit#157], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#155, returns#156, profit#157] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(94) CometFilter +(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)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] 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 = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 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-spark4_0/q5a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt index 601bfabb85..35b86dbff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/extended.txt @@ -1,283 +1,269 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - : +- CometColumnarToRow - : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - : +- CometColumnarToRow - : +- CometExchange - : +- 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 : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate + : +- 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 - +- 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 + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 218 out of 263 eligible operators (82%). Final plan contains 18 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-spark4_0/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a.native_iceberg_compat/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - 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 (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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] - WholeStageCodegen (2) - 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 - 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] - WholeStageCodegen (3) - 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 - 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] - WholeStageCodegen (11) - 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] #12 - WholeStageCodegen (10) - 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 (17) - 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 #13 - WholeStageCodegen (16) - 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-spark4_0/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt index a3c14dc037..ca9ea23933 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (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) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* 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 @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV 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] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(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))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(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) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(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) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(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] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(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] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(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) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(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#62, cp_catalog_page_id#63] +(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#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#63, 16)) AS cp_catalog_page_id#64] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#54, 16)) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(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] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(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 -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(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] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(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))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(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] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(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))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(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#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(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) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(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] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(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 -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(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)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(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] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(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 -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#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] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#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] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(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 -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(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 -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#88, 16)) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#106, 16)) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(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 -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], 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] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(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]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -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#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [2]: [channel#139, id#140] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#133, sum(returns#148)#134, sum(profit#149)#135] -Results [4]: [channel#139, sum(sales#147)#133 AS sales#150, sum(returns#148)#134 AS returns#151, sum(profit#149)#135 AS profit#152] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#139, sales#150, returns#151, profit#152] -Keys [1]: [channel#139] -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#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(77) CometColumnarExchange -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: hashpartitioning(channel#139, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#139, sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Keys [1]: [channel#139] -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#139, 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] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#172, id#173, sum#174, isEmpty#175, sum#176, isEmpty#177, sum#178, isEmpty#179] -Keys [2]: [channel#172, id#173] -Functions [3]: [sum(sales#180), sum(returns#181), sum(profit#182)] -Aggregate Attributes [3]: [sum(sales#180)#133, sum(returns#181)#134, sum(profit#182)#135] -Results [3]: [sum(sales#180)#133 AS sales#183, sum(returns#181)#134 AS returns#184, sum(profit#182)#135 AS profit#185] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#183, returns#184, profit#185] +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#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(70) CometHashAggregate +Input [8]: [channel#115, id#116, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [2]: [channel#115, id#116] +Functions [3]: [sum(sales#123), sum(returns#124), sum(profit#125)] + +(71) CometHashAggregate +Input [4]: [channel#115, sales#126, returns#127, profit#128] +Keys [1]: [channel#115] +Functions [3]: [partial_sum(sales#126), partial_sum(returns#127), partial_sum(profit#128)] + +(72) CometExchange +Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Arguments: hashpartitioning(channel#115, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#115, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [1]: [channel#115] +Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] + +(75) CometHashAggregate +Input [8]: [channel#135, id#136, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Keys [2]: [channel#135, id#136] +Functions [3]: [sum(sales#143), sum(returns#144), sum(profit#145)] + +(76) CometHashAggregate +Input [3]: [sales#146, returns#147, profit#148] Keys: [] -Functions [3]: [partial_sum(sales#183), partial_sum(returns#184), partial_sum(profit#185)] -Aggregate Attributes [6]: [sum#186, isEmpty#187, sum#188, isEmpty#189, sum#190, isEmpty#191] -Results [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] - -(84) CometColumnarExchange -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#146), partial_sum(returns#147), partial_sum(profit#148)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +(77) CometExchange +Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197] +(78) CometHashAggregate +Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] Keys: [] -Functions [3]: [sum(sales#183), sum(returns#184), sum(profit#185)] -Aggregate Attributes [3]: [sum(sales#183)#198, sum(returns#184)#199, sum(profit#185)#200] -Results [5]: [null AS channel#201, null AS id#202, sum(sales#183)#198 AS sum(sales)#203, sum(returns#184)#199 AS sum(returns)#204, sum(profit#185)#200 AS sum(profit)#205] +Functions [3]: [sum(sales#146), sum(returns#147), sum(profit#148)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Child 1 Input [5]: [channel#115, id#158, sum(sales)#159, sum(returns)#160, sum(profit)#161] +Child 2 Input [5]: [channel#162, id#163, sum(sales)#164, sum(returns)#165, sum(profit)#166] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Arguments: hashpartitioning(channel#94, id#95, sales#155, returns#156, profit#157, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Keys [5]: [channel#94, id#95, sales#155, returns#156, profit#157] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#155,returns#156,profit#157]), [channel#94, id#95, sales#155, returns#156, profit#157], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#155, returns#156, profit#157] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#155, returns#156, profit#157] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(94) CometFilter +(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)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] 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 = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 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-spark4_0/q5a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt index 601bfabb85..35b86dbff5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/extended.txt @@ -1,283 +1,269 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - : +- CometColumnarToRow - : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - : +- CometColumnarToRow - : +- CometExchange - : +- 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 : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(sales_price#4))#5,17,2) AS sales#6, MakeDecimal(sum(UnscaledValue(return_amt#7))#8,17,2) AS returns#9, (MakeDecimal(sum(UnscaledValue(profit#10))#11,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#12))#13,17,2)) AS profit#14)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#15, concat(catalog_page, cp_catalog_page_id#16) AS id#17, MakeDecimal(sum(UnscaledValue(sales_price#18))#19,17,2) AS sales#20, MakeDecimal(sum(UnscaledValue(return_amt#21))#22,17,2) AS returns#23, (MakeDecimal(sum(UnscaledValue(profit#24))#25,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#26))#27,17,2)) AS profit#28)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#29, concat(web_site, web_site_id#30) AS id#31, MakeDecimal(sum(UnscaledValue(sales_price#32))#33,17,2) AS sales#34, MakeDecimal(sum(UnscaledValue(return_amt#35))#36,17,2) AS returns#37, (MakeDecimal(sum(UnscaledValue(profit#38))#39,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#40))#41,17,2)) AS profit#42)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate + : +- 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 - +- 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 + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Comet accelerated 218 out of 263 eligible operators (82%). Final plan contains 18 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-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - 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 (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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] - WholeStageCodegen (2) - 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 - 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] - WholeStageCodegen (3) - 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 - 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] - WholeStageCodegen (11) - 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] #12 - WholeStageCodegen (10) - 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 (17) - 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 #13 - WholeStageCodegen (16) - 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-spark4_0/q64.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt index e2784d3e5b..8764db8779 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/explain.txt @@ -1,189 +1,180 @@ == Physical Plan == -* CometColumnarToRow (185) -+- CometSort (184) - +- CometExchange (183) - +- CometProject (182) - +- CometSortMergeJoin (181) - :- CometSort (112) - : +- CometColumnarExchange (111) - : +- * HashAggregate (110) - : +- * HashAggregate (109) - : +- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- * Project (92) - : : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : : :- * Project (89) - : : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : : :- * Project (82) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : : :- * Project (79) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : : :- * Project (73) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) - : : : : : : : : :- * Project (67) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : : : : : : : :- * Project (64) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) - : : : : : : : : : : : : : : : : : +- 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) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (26) - : : : : : : : : : : : : : : : : +- CometExchange (25) - : : : : : : : : : : : : : : : : +- CometHashAggregate (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * CometColumnarToRow (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * CometColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * CometColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (62) - : : : : : : : : : : +- * CometColumnarToRow (61) - : : : : : : : : : : +- CometProject (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (65) - : : : : : : : : +- BroadcastExchange (71) - : : : : : : : : +- * CometColumnarToRow (70) - : : : : : : : : +- CometFilter (69) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (68) - : : : : : : : +- BroadcastExchange (77) - : : : : : : : +- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (74) - : : : : : : +- ReusedExchange (80) - : : : : : +- BroadcastExchange (87) - : : : : : +- * CometColumnarToRow (86) - : : : : : +- CometProject (85) - : : : : : +- CometFilter (84) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (83) - : : : : +- ReusedExchange (90) - : : : +- BroadcastExchange (96) - : : : +- * CometColumnarToRow (95) - : : : +- CometFilter (94) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (93) - : : +- ReusedExchange (99) - : +- BroadcastExchange (106) - : +- * CometColumnarToRow (105) - : +- CometProject (104) - : +- CometFilter (103) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (102) - +- CometSort (180) - +- CometColumnarExchange (179) - +- * HashAggregate (178) - +- * HashAggregate (177) - +- * Project (176) - +- * BroadcastHashJoin Inner BuildRight (175) - :- * Project (173) - : +- * BroadcastHashJoin Inner BuildRight (172) - : :- * Project (170) - : : +- * BroadcastHashJoin Inner BuildRight (169) - : : :- * Project (167) - : : : +- * BroadcastHashJoin Inner BuildRight (166) - : : : :- * Project (164) - : : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : : :- * Project (161) - : : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : : :- * Project (158) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : : :- * Project (155) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : : :- * Project (152) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : : :- * 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) - : : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (130) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (123) - : : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : : +- CometExchange (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (119) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (115) - : : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (113) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometFilter (117) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (116) - : : : : : : : : : : : : : : : +- * Sort (129) - : : : : : : : : : : : : : : : +- * Project (128) - : : : : : : : : : : : : : : : +- * Filter (127) - : : : : : : : : : : : : : : : +- * HashAggregate (126) - : : : : : : : : : : : : : : : +- * CometColumnarToRow (125) - : : : : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : : +- ReusedExchange (147) - : : : : : : : : +- ReusedExchange (150) - : : : : : : : +- ReusedExchange (153) - : : : : : : +- ReusedExchange (156) - : : : : : +- ReusedExchange (159) - : : : : +- ReusedExchange (162) - : : : +- ReusedExchange (165) - : : +- ReusedExchange (168) - : +- ReusedExchange (171) - +- ReusedExchange (174) +* 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 @@ -234,846 +225,787 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) CometColumnarToRow [codegen id : 1] -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] - -(12) 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 -(13) 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)) -(14) 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] -(15) 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=2] -(16) 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] -(17) 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 -(18) 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)) -(19) 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] -(20) 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=3] -(21) 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] -(22) 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 -(23) 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] -(24) 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))] -(25) 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=4] -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] - -(27) HashAggregate [codegen id : 2] +(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))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#30, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#30,17,2) AS sale#32, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31 AS refund#33] -(28) Filter [codegen id : 2] -Input [3]: [cs_item_sk#17, sale#32, refund#33] -Condition : ((isnotnull(sale#32) AND isnotnull(refund#33)) AND (cast(sale#32 as decimal(21,2)) > (2 * refund#33))) +(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) Project [codegen id : 2] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#32, refund#33] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) Sort [codegen id : 2] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) SortMergeJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(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) 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, ss_sold_date_sk#12] +(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) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#34, d_year#35] +(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)) -(34) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] -(35) 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#35] -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#34, d_year#35] +(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] (36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#36, s_store_name#37, s_zip#38] +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 -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Condition : ((isnotnull(s_store_sk#36) AND isnotnull(s_store_name#37)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)))) +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(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) (38) CometProject -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Arguments: [s_store_sk#36, s_store_name#37, s_zip#39], [s_store_sk#36, s_store_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)) AS s_zip#39] - -(39) CometColumnarToRow [codegen id : 4] -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] +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, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] -(40) BroadcastExchange -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#36] -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 -(42) 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#35, s_store_name#37, s_zip#39] -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#35, s_store_sk#36, s_store_name#37, s_zip#39] +(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] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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 -(44) CometFilter -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Condition : (((((isnotnull(c_customer_sk#40) AND isnotnull(c_first_sales_date_sk#45)) AND isnotnull(c_first_shipto_date_sk#44)) AND isnotnull(c_current_cdemo_sk#41)) AND isnotnull(c_current_hdemo_sk#42)) AND isnotnull(c_current_addr_sk#43)) - -(45) CometColumnarToRow [codegen id : 5] -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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) BroadcastExchange -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#40] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -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#35, s_store_name#37, s_zip#39, c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] +(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 -(50) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : isnotnull(d_date_sk#46) - -(51) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#46, d_year#47] +(48) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : isnotnull(d_date_sk#44) -(52) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(49) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] -(53) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#45] -Right keys [1]: [d_date_sk#46] -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 -(54) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45, d_date_sk#46, d_year#47] +(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] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#48, d_year#49] +(52) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#46, d_year#47] -(56) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#44] -Right keys [1]: [d_date_sk#48] -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 -(57) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, d_year#47, d_date_sk#48, d_year#49] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#50, cd_marital_status#51] +(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 -(59) CometFilter -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Condition : (isnotnull(cd_demo_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)))) +(56) CometFilter +Input [2]: [cd_demo_sk#48, cd_marital_status#49] +Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) -(60) CometProject -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Arguments: [cd_demo_sk#50, cd_marital_status#52], [cd_demo_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)) AS cd_marital_status#52] - -(61) CometColumnarToRow [codegen id : 8] -Input [2]: [cd_demo_sk#50, cd_marital_status#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] -(62) BroadcastExchange -Input [2]: [cd_demo_sk#50, cd_marital_status#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) CometBroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#50] -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 -(64) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_demo_sk#50, cd_marital_status#52] +(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] -(65) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#53, cd_marital_status#54] +(61) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#41] -Right keys [1]: [cd_demo_sk#53] -Join type: Inner -Join condition: NOT (cd_marital_status#52 = cd_marital_status#54) +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52, cd_demo_sk#53, cd_marital_status#54] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#55] +(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 -(69) CometFilter -Input [1]: [p_promo_sk#55] -Condition : isnotnull(p_promo_sk#55) +(65) CometFilter +Input [1]: [p_promo_sk#53] +Condition : isnotnull(p_promo_sk#53) -(70) CometColumnarToRow [codegen id : 10] -Input [1]: [p_promo_sk#55] +(66) CometBroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(71) BroadcastExchange -Input [1]: [p_promo_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#55] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, p_promo_sk#55] +(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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(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 -(75) CometFilter -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Condition : (isnotnull(hd_demo_sk#56) AND isnotnull(hd_income_band_sk#57)) +(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)) -(76) CometColumnarToRow [codegen id : 11] -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(71) CometBroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(77) BroadcastExchange -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#56] -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 -(79) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_demo_sk#56, hd_income_band_sk#57] +(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] -(80) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#58, hd_income_band_sk#59] +(74) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#42] -Right keys [1]: [hd_demo_sk#58] -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 -(82) 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#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_demo_sk#58, hd_income_band_sk#59] +(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] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] +(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 -(84) CometFilter -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Condition : isnotnull(ca_address_sk#60) +(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) -(85) CometProject -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Arguments: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66], [ca_address_sk#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#61, 10)) AS ca_street_number#65, ca_street_name#62, ca_city#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#64, 10)) AS ca_zip#66] - -(86) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] -(87) BroadcastExchange -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(80) CometBroadcastExchange +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] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#60] -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 -(89) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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] -(90) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(91) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#67] -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 -(92) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#72] +(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 -(94) CometFilter -Input [1]: [ib_income_band_sk#72] -Condition : isnotnull(ib_income_band_sk#72) +(87) CometFilter +Input [1]: [ib_income_band_sk#70] +Condition : isnotnull(ib_income_band_sk#70) -(95) CometColumnarToRow [codegen id : 15] -Input [1]: [ib_income_band_sk#72] +(88) CometBroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(96) BroadcastExchange -Input [1]: [ib_income_band_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(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] -(98) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#72] +(91) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#71] -(99) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#73] +(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 -(100) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#59] -Right keys [1]: [ib_income_band_sk#73] -Join type: Inner -Join condition: None +(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] -(101) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#73] - -(102) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] +(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 -(103) CometFilter -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Condition : ((((((isnotnull(i_current_price#75) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#76, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#75 >= 64.00)) AND (i_current_price#75 <= 74.00)) AND (i_current_price#75 >= 65.00)) AND (i_current_price#75 <= 79.00)) AND isnotnull(i_item_sk#74)) - -(104) CometProject -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Arguments: [i_item_sk#74, i_product_name#78], [i_item_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#77, 50)) AS i_product_name#78] +(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 static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) 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) CometColumnarToRow [codegen id : 17] -Input [2]: [i_item_sk#74, i_product_name#78] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] -(106) BroadcastExchange -Input [2]: [i_item_sk#74, i_product_name#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(97) CometBroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#74] -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 -(108) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] +(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] -(109) HashAggregate [codegen id : 18] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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#79, sum#80, sum#81, sum#82] -Results [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -(110) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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)#87, sum(UnscaledValue(ss_wholesale_cost#9))#88, sum(UnscaledValue(ss_list_price#10))#89, sum(UnscaledValue(ss_coupon_amt#11))#90] -Results [17]: [i_product_name#78 AS product_name#91, i_item_sk#74 AS item_sk#92, s_store_name#37 AS store_name#93, s_zip#39 AS store_zip#94, ca_street_number#65 AS b_street_number#95, ca_street_name#62 AS b_streen_name#96, ca_city#63 AS b_city#97, ca_zip#66 AS b_zip#98, ca_street_number#68 AS c_street_number#99, ca_street_name#69 AS c_street_name#100, ca_city#70 AS c_city#101, ca_zip#71 AS c_zip#102, d_year#35 AS syear#103, count(1)#87 AS cnt#104, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#88,17,2) AS s1#105, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#89,17,2) AS s2#106, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#90,17,2) AS s3#107] -(111) CometColumnarExchange -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: hashpartitioning(item_sk#92, store_name#93, store_zip#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(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) CometSort -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107], [item_sk#92 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, store_zip#94 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] -(113) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] +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 -(114) CometFilter -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Condition : (((((((isnotnull(ss_item_sk#108) AND isnotnull(ss_ticket_number#115)) AND isnotnull(ss_store_sk#113)) AND isnotnull(ss_customer_sk#109)) AND isnotnull(ss_cdemo_sk#110)) AND isnotnull(ss_promo_sk#114)) AND isnotnull(ss_hdemo_sk#111)) AND isnotnull(ss_addr_sk#112)) +(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) CometBroadcastExchange -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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] -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] +(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 -(117) CometFilter -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Condition : (isnotnull(sr_item_sk#121) AND isnotnull(sr_ticket_number#122)) +(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))] -(118) CometProject -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Arguments: [sr_item_sk#121, sr_ticket_number#122], [sr_item_sk#121, sr_ticket_number#122] +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#122, refund#123] +Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) -(119) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Right output [2]: [sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_ticket_number#115], [sr_item_sk#121, sr_ticket_number#122], Inner, BuildLeft +(117) CometProject +Input [3]: [cs_item_sk#114, sale#122, refund#123] +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 [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(121) CometExchange -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: hashpartitioning(ss_item_sk#108, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(122) CometSort -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108 ASC NULLS FIRST] - -(123) CometColumnarToRow [codegen id : 19] -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(124) ReusedExchange [Reuses operator id: 25] -Output [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(125) CometColumnarToRow [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(126) HashAggregate [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#124] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#128)), sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#128))#30, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31] -Results [3]: [cs_item_sk#124, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#128))#30,17,2) AS sale#132, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31 AS refund#133] - -(127) Filter [codegen id : 20] -Input [3]: [cs_item_sk#124, sale#132, refund#133] -Condition : ((isnotnull(sale#132) AND isnotnull(refund#133)) AND (cast(sale#132 as decimal(21,2)) > (2 * refund#133))) - -(128) Project [codegen id : 20] -Output [1]: [cs_item_sk#124] -Input [3]: [cs_item_sk#124, sale#132, refund#133] - -(129) Sort [codegen id : 20] -Input [1]: [cs_item_sk#124] -Arguments: [cs_item_sk#124 ASC NULLS FIRST], false, 0 - -(130) SortMergeJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [cs_item_sk#124] -Join type: Inner -Join condition: None - -(131) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, cs_item_sk#124] - -(132) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#134, d_year#135] - -(133) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#119] -Right keys [1]: [d_date_sk#134] -Join type: Inner -Join condition: None - -(134) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135] -Input [13]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, d_date_sk#134, d_year#135] - -(135) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#136, s_store_name#137, s_zip#138] - -(136) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#113] -Right keys [1]: [s_store_sk#136] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 36] -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138] -Input [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_sk#136, s_store_name#137, s_zip#138] - -(138) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(139) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#109] -Right keys [1]: [c_customer_sk#139] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] -Input [18]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#145, d_year#146] - -(142) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#144] -Right keys [1]: [d_date_sk#145] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144, d_date_sk#145, d_year#146] - -(144) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#147, d_year#148] - -(145) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#143] -Right keys [1]: [d_date_sk#147] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146, d_date_sk#147, d_year#148] - -(147) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#149, cd_marital_status#150] - -(148) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#110] -Right keys [1]: [cd_demo_sk#149] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_demo_sk#149, cd_marital_status#150] - -(150) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#151, cd_marital_status#152] - -(151) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#140] -Right keys [1]: [cd_demo_sk#151] -Join type: Inner -Join condition: NOT (cd_marital_status#150 = cd_marital_status#152) - -(152) Project [codegen id : 36] -Output [14]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150, cd_demo_sk#151, cd_marital_status#152] - -(153) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#153] - -(154) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#114] -Right keys [1]: [p_promo_sk#153] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, p_promo_sk#153] - -(156) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] - -(157) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#111] -Right keys [1]: [hd_demo_sk#154] -Join type: Inner -Join condition: None - -(158) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_demo_sk#154, hd_income_band_sk#155] - -(159) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#156, hd_income_band_sk#157] - -(160) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#141] -Right keys [1]: [hd_demo_sk#156] -Join type: Inner -Join condition: None - -(161) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157] -Input [15]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_demo_sk#156, hd_income_band_sk#157] - -(162) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(163) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#112] -Right keys [1]: [ca_address_sk#158] -Join type: Inner -Join condition: None - -(164) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] -Input [18]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(165) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(166) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#142] -Right keys [1]: [ca_address_sk#163] -Join type: Inner -Join condition: None - -(167) Project [codegen id : 36] -Output [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [21]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(168) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#168] - -(169) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#155] -Right keys [1]: [ib_income_band_sk#168] -Join type: Inner -Join condition: None - -(170) Project [codegen id : 36] -Output [18]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [20]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#168] - -(171) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#169] - -(172) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#157] -Right keys [1]: [ib_income_band_sk#169] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 36] -Output [17]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#169] - -(174) ReusedExchange [Reuses operator id: 106] -Output [2]: [i_item_sk#170, i_product_name#171] - -(175) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [i_item_sk#170] -Join type: Inner -Join condition: None - -(176) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] - -(177) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#116)), partial_sum(UnscaledValue(ss_list_price#117)), partial_sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count#79, sum#172, sum#173, sum#174] -Results [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] - -(178) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#116)), sum(UnscaledValue(ss_list_price#117)), sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count(1)#87, sum(UnscaledValue(ss_wholesale_cost#116))#88, sum(UnscaledValue(ss_list_price#117))#89, sum(UnscaledValue(ss_coupon_amt#118))#90] -Results [8]: [i_item_sk#170 AS item_sk#178, s_store_name#137 AS store_name#179, s_zip#138 AS store_zip#180, d_year#135 AS syear#181, count(1)#87 AS cnt#182, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#116))#88,17,2) AS s1#183, MakeDecimal(sum(UnscaledValue(ss_list_price#117))#89,17,2) AS s2#184, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#118))#90,17,2) AS s3#185] - -(179) CometColumnarExchange -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: hashpartitioning(item_sk#178, store_name#179, store_zip#180, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(180) CometSort -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185], [item_sk#178 ASC NULLS FIRST, store_name#179 ASC NULLS FIRST, store_zip#180 ASC NULLS FIRST] - -(181) CometSortMergeJoin -Left output [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Right output [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#179, store_zip#180], Inner, (cnt#182 <= cnt#104) - -(182) CometProject -Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] - -(183) CometExchange -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] - -(184) CometSort -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST] - -(185) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] +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#124, d_year#125] +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#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) + +(123) CometBroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +Arguments: [d_date_sk#124, d_year#125] + +(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#124, d_year#125] +Arguments: [ss_sold_date_sk#109], [d_date_sk#124], 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#124, d_year#125] +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#125], [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#125] + +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] + +(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#125] +Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] +Arguments: [ss_store_sk#103], [s_store_sk#126], 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#125, s_store_sk#126, s_store_name#127, s_zip#128] +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#125, s_store_name#127, s_zip#128], [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#125, s_store_name#127, s_zip#128] + +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(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#125, s_store_name#127, s_zip#128] +Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Arguments: [ss_customer_sk#99], [c_customer_sk#129], 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#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#135, d_year#136] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Right output [2]: [d_date_sk#135, d_year#136] +Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#137, d_year#138] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] +Right output [2]: [d_date_sk#137, d_year#138] +Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#141, cd_marital_status#142] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] +Right output [2]: [cd_demo_sk#141, cd_marital_status#142] +Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#143] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [1]: [p_promo_sk#143] +Arguments: [ss_promo_sk#104], [p_promo_sk#143], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] +Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] +Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_addr_sk#102], [ca_address_sk#148], 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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#158] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#158] +Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#159] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#159] +Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#160, i_product_name#161] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [2]: [i_item_sk#160, i_product_name#161] +Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +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#170, s2#171, s3#172, syear#168, cnt#169], [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#170, s2#171, s3#172, syear#168, cnt#169] + +(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#170, s2#171, s3#172, syear#168, cnt#169] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 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#170, s2#171, s3#172, syear#168, cnt#169] +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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * CometColumnarToRow (188) - +- CometFilter (187) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (186) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(186) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +(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 -(187) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 1999)) AND isnotnull(d_date_sk#34)) +(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)) -(188) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(179) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(189) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 113 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#120 -BroadcastExchange (193) -+- * CometColumnarToRow (192) - +- CometFilter (191) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (190) +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) -(190) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#134, d_year#135] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#124, d_year#125] 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 -(191) CometFilter -Input [2]: [d_date_sk#134, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND isnotnull(d_date_sk#134)) +(182) CometFilter +Input [2]: [d_date_sk#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(192) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#134, d_year#135] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#124, d_year#125] -(193) BroadcastExchange -Input [2]: [d_date_sk#134, d_year#135] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +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-spark4_0/q64.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt index 41817bd434..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/extended.txt @@ -4,278 +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 - : : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#1, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#2))#3,17,2) AS sale#4, sum(((cr_refunded_cash#5 + cr_reversed_charge#6) + cr_store_credit#7))#8 AS refund#9)] - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#10, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#11))#3,17,2) AS sale#12, sum(((cr_refunded_cash#13 + cr_reversed_charge#14) + cr_store_credit#15))#8 AS refund#16)] - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- 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 132 out of 242 eligible operators (54%). Final plan contains 37 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt index 29748bfd47..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/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,258 +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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] - InputAdapter - WholeStageCodegen (2) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - 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] #19 - 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] #20 - 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] - 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 - WholeStageCodegen (20) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt index e2784d3e5b..8764db8779 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -1,189 +1,180 @@ == Physical Plan == -* CometColumnarToRow (185) -+- CometSort (184) - +- CometExchange (183) - +- CometProject (182) - +- CometSortMergeJoin (181) - :- CometSort (112) - : +- CometColumnarExchange (111) - : +- * HashAggregate (110) - : +- * HashAggregate (109) - : +- * Project (108) - : +- * BroadcastHashJoin Inner BuildRight (107) - : :- * Project (101) - : : +- * BroadcastHashJoin Inner BuildRight (100) - : : :- * Project (98) - : : : +- * BroadcastHashJoin Inner BuildRight (97) - : : : :- * Project (92) - : : : : +- * BroadcastHashJoin Inner BuildRight (91) - : : : : :- * Project (89) - : : : : : +- * BroadcastHashJoin Inner BuildRight (88) - : : : : : :- * Project (82) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (81) - : : : : : : :- * Project (79) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (78) - : : : : : : : :- * Project (73) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (72) - : : : : : : : : :- * Project (67) - : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (66) - : : : : : : : : : :- * Project (64) - : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (63) - : : : : : : : : : : :- * Project (57) - : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) - : : : : : : : : : : : :- * Project (54) - : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) - : : : : : : : : : : : : :- * Project (48) - : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) - : : : : : : : : : : : : : :- * Project (42) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) - : : : : : : : : : : : : : : :- * Project (35) - : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) - : : : : : : : : : : : : : : : :- * Project (32) - : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (31) - : : : : : : : : : : : : : : : : :- * CometColumnarToRow (11) - : : : : : : : : : : : : : : : : : +- 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) - : : : : : : : : : : : : : : : : +- * Sort (30) - : : : : : : : : : : : : : : : : +- * Project (29) - : : : : : : : : : : : : : : : : +- * Filter (28) - : : : : : : : : : : : : : : : : +- * HashAggregate (27) - : : : : : : : : : : : : : : : : +- * CometColumnarToRow (26) - : : : : : : : : : : : : : : : : +- CometExchange (25) - : : : : : : : : : : : : : : : : +- CometHashAggregate (24) - : : : : : : : : : : : : : : : : +- CometProject (23) - : : : : : : : : : : : : : : : : +- CometSortMergeJoin (22) - : : : : : : : : : : : : : : : : :- CometSort (16) - : : : : : : : : : : : : : : : : : +- CometExchange (15) - : : : : : : : : : : : : : : : : : +- CometProject (14) - : : : : : : : : : : : : : : : : : +- CometFilter (13) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (12) - : : : : : : : : : : : : : : : : +- CometSort (21) - : : : : : : : : : : : : : : : : +- CometExchange (20) - : : : : : : : : : : : : : : : : +- CometProject (19) - : : : : : : : : : : : : : : : : +- CometFilter (18) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (17) - : : : : : : : : : : : : : : : +- ReusedExchange (33) - : : : : : : : : : : : : : : +- BroadcastExchange (40) - : : : : : : : : : : : : : : +- * CometColumnarToRow (39) - : : : : : : : : : : : : : : +- CometProject (38) - : : : : : : : : : : : : : : +- CometFilter (37) - : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (36) - : : : : : : : : : : : : : +- BroadcastExchange (46) - : : : : : : : : : : : : : +- * CometColumnarToRow (45) - : : : : : : : : : : : : : +- CometFilter (44) - : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (43) - : : : : : : : : : : : : +- BroadcastExchange (52) - : : : : : : : : : : : : +- * CometColumnarToRow (51) - : : : : : : : : : : : : +- CometFilter (50) - : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) - : : : : : : : : : : : +- ReusedExchange (55) - : : : : : : : : : : +- BroadcastExchange (62) - : : : : : : : : : : +- * CometColumnarToRow (61) - : : : : : : : : : : +- CometProject (60) - : : : : : : : : : : +- CometFilter (59) - : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics (58) - : : : : : : : : : +- ReusedExchange (65) - : : : : : : : : +- BroadcastExchange (71) - : : : : : : : : +- * CometColumnarToRow (70) - : : : : : : : : +- CometFilter (69) - : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (68) - : : : : : : : +- BroadcastExchange (77) - : : : : : : : +- * CometColumnarToRow (76) - : : : : : : : +- CometFilter (75) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics (74) - : : : : : : +- ReusedExchange (80) - : : : : : +- BroadcastExchange (87) - : : : : : +- * CometColumnarToRow (86) - : : : : : +- CometProject (85) - : : : : : +- CometFilter (84) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address (83) - : : : : +- ReusedExchange (90) - : : : +- BroadcastExchange (96) - : : : +- * CometColumnarToRow (95) - : : : +- CometFilter (94) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band (93) - : : +- ReusedExchange (99) - : +- BroadcastExchange (106) - : +- * CometColumnarToRow (105) - : +- CometProject (104) - : +- CometFilter (103) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (102) - +- CometSort (180) - +- CometColumnarExchange (179) - +- * HashAggregate (178) - +- * HashAggregate (177) - +- * Project (176) - +- * BroadcastHashJoin Inner BuildRight (175) - :- * Project (173) - : +- * BroadcastHashJoin Inner BuildRight (172) - : :- * Project (170) - : : +- * BroadcastHashJoin Inner BuildRight (169) - : : :- * Project (167) - : : : +- * BroadcastHashJoin Inner BuildRight (166) - : : : :- * Project (164) - : : : : +- * BroadcastHashJoin Inner BuildRight (163) - : : : : :- * Project (161) - : : : : : +- * BroadcastHashJoin Inner BuildRight (160) - : : : : : :- * Project (158) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (157) - : : : : : : :- * Project (155) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) - : : : : : : : :- * Project (152) - : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) - : : : : : : : : :- * 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) - : : : : : : : : : : : : : :- * Project (134) - : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) - : : : : : : : : : : : : : : :- * Project (131) - : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (130) - : : : : : : : : : : : : : : : :- * CometColumnarToRow (123) - : : : : : : : : : : : : : : : : +- CometSort (122) - : : : : : : : : : : : : : : : : +- CometExchange (121) - : : : : : : : : : : : : : : : : +- CometProject (120) - : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (119) - : : : : : : : : : : : : : : : : :- CometBroadcastExchange (115) - : : : : : : : : : : : : : : : : : +- CometFilter (114) - : : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (113) - : : : : : : : : : : : : : : : : +- CometProject (118) - : : : : : : : : : : : : : : : : +- CometFilter (117) - : : : : : : : : : : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (116) - : : : : : : : : : : : : : : : +- * Sort (129) - : : : : : : : : : : : : : : : +- * Project (128) - : : : : : : : : : : : : : : : +- * Filter (127) - : : : : : : : : : : : : : : : +- * HashAggregate (126) - : : : : : : : : : : : : : : : +- * CometColumnarToRow (125) - : : : : : : : : : : : : : : : +- ReusedExchange (124) - : : : : : : : : : : : : : : +- ReusedExchange (132) - : : : : : : : : : : : : : +- ReusedExchange (135) - : : : : : : : : : : : : +- ReusedExchange (138) - : : : : : : : : : : : +- ReusedExchange (141) - : : : : : : : : : : +- ReusedExchange (144) - : : : : : : : : : +- ReusedExchange (147) - : : : : : : : : +- ReusedExchange (150) - : : : : : : : +- ReusedExchange (153) - : : : : : : +- ReusedExchange (156) - : : : : : +- ReusedExchange (159) - : : : : +- ReusedExchange (162) - : : : +- ReusedExchange (165) - : : +- ReusedExchange (168) - : +- ReusedExchange (171) - +- ReusedExchange (174) +* 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 @@ -234,846 +225,787 @@ Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh 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) CometColumnarToRow [codegen id : 1] -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] - -(12) 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 -(13) 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)) -(14) 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] -(15) 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=2] -(16) 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] -(17) 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 -(18) 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)) -(19) 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] -(20) 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=3] -(21) 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] -(22) 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 -(23) 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] -(24) 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))] -(25) 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=4] -(26) CometColumnarToRow [codegen id : 2] -Input [4]: [cs_item_sk#17, sum#27, sum#28, isEmpty#29] - -(27) HashAggregate [codegen id : 2] +(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))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#30, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31] -Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#30,17,2) AS sale#32, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#31 AS refund#33] -(28) Filter [codegen id : 2] -Input [3]: [cs_item_sk#17, sale#32, refund#33] -Condition : ((isnotnull(sale#32) AND isnotnull(refund#33)) AND (cast(sale#32 as decimal(21,2)) > (2 * refund#33))) +(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) Project [codegen id : 2] -Output [1]: [cs_item_sk#17] -Input [3]: [cs_item_sk#17, sale#32, refund#33] +(27) CometProject +Input [3]: [cs_item_sk#17, sale#30, refund#31] +Arguments: [cs_item_sk#17], [cs_item_sk#17] -(30) Sort [codegen id : 2] +(28) CometSort Input [1]: [cs_item_sk#17] -Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 +Arguments: [cs_item_sk#17], [cs_item_sk#17 ASC NULLS FIRST] -(31) SortMergeJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [cs_item_sk#17] -Join type: Inner -Join condition: None +(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) 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, ss_sold_date_sk#12] +(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) ReusedExchange [Reuses operator id: 189] -Output [2]: [d_date_sk#34, d_year#35] +(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)) -(34) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#12] -Right keys [1]: [d_date_sk#34] -Join type: Inner -Join condition: None +(33) CometBroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: [d_date_sk#32, d_year#33] -(35) 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#35] -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#34, d_year#35] +(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] (36) CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Output [3]: [s_store_sk#36, s_store_name#37, s_zip#38] +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 -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Condition : ((isnotnull(s_store_sk#36) AND isnotnull(s_store_name#37)) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)))) +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(static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)))) (38) CometProject -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#38] -Arguments: [s_store_sk#36, s_store_name#37, s_zip#39], [s_store_sk#36, s_store_name#37, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#38, 10)) AS s_zip#39] - -(39) CometColumnarToRow [codegen id : 4] -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] +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, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_zip#36, 10)) AS s_zip#37] -(40) BroadcastExchange -Input [3]: [s_store_sk#36, s_store_name#37, s_zip#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_store_sk#6] -Right keys [1]: [s_store_sk#36] -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 -(42) 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#35, s_store_name#37, s_zip#39] -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#35, s_store_sk#36, s_store_name#37, s_zip#39] +(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] -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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 -(44) CometFilter -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Condition : (((((isnotnull(c_customer_sk#40) AND isnotnull(c_first_sales_date_sk#45)) AND isnotnull(c_first_shipto_date_sk#44)) AND isnotnull(c_current_cdemo_sk#41)) AND isnotnull(c_current_hdemo_sk#42)) AND isnotnull(c_current_addr_sk#43)) - -(45) CometColumnarToRow [codegen id : 5] -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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) BroadcastExchange -Input [6]: [c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#40] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] -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#35, s_store_name#37, s_zip#39, c_customer_sk#40, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45] +(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] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#46, d_year#47] +(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 -(50) CometFilter -Input [2]: [d_date_sk#46, d_year#47] -Condition : isnotnull(d_date_sk#46) - -(51) CometColumnarToRow [codegen id : 6] -Input [2]: [d_date_sk#46, d_year#47] +(48) CometFilter +Input [2]: [d_date_sk#44, d_year#45] +Condition : isnotnull(d_date_sk#44) -(52) BroadcastExchange -Input [2]: [d_date_sk#46, d_year#47] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] +(49) CometBroadcastExchange +Input [2]: [d_date_sk#44, d_year#45] +Arguments: [d_date_sk#44, d_year#45] -(53) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_sales_date_sk#45] -Right keys [1]: [d_date_sk#46] -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 -(54) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, c_first_sales_date_sk#45, d_date_sk#46, d_year#47] +(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] -(55) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#48, d_year#49] +(52) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#46, d_year#47] -(56) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_first_shipto_date_sk#44] -Right keys [1]: [d_date_sk#48] -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 -(57) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, c_first_shipto_date_sk#44, d_year#47, d_date_sk#48, d_year#49] +(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] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_demographics -Output [2]: [cd_demo_sk#50, cd_marital_status#51] +(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 -(59) CometFilter -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Condition : (isnotnull(cd_demo_sk#50) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)))) +(56) CometFilter +Input [2]: [cd_demo_sk#48, cd_marital_status#49] +Condition : (isnotnull(cd_demo_sk#48) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)))) -(60) CometProject -Input [2]: [cd_demo_sk#50, cd_marital_status#51] -Arguments: [cd_demo_sk#50, cd_marital_status#52], [cd_demo_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#51, 1)) AS cd_marital_status#52] - -(61) CometColumnarToRow [codegen id : 8] -Input [2]: [cd_demo_sk#50, cd_marital_status#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cd_marital_status#49, 1)) AS cd_marital_status#50] -(62) BroadcastExchange -Input [2]: [cd_demo_sk#50, cd_marital_status#52] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] +(58) CometBroadcastExchange +Input [2]: [cd_demo_sk#48, cd_marital_status#50] +Arguments: [cd_demo_sk#48, cd_marital_status#50] -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#50] -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 -(64) 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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_demo_sk#50, cd_marital_status#52] +(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] -(65) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#53, cd_marital_status#54] +(61) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#51, cd_marital_status#52] -(66) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_cdemo_sk#41] -Right keys [1]: [cd_demo_sk#53] -Join type: Inner -Join condition: NOT (cd_marital_status#52 = cd_marital_status#54) +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_cdemo_sk#41, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, cd_marital_status#52, cd_demo_sk#53, cd_marital_status#54] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Output [1]: [p_promo_sk#55] +(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 -(69) CometFilter -Input [1]: [p_promo_sk#55] -Condition : isnotnull(p_promo_sk#55) +(65) CometFilter +Input [1]: [p_promo_sk#53] +Condition : isnotnull(p_promo_sk#53) -(70) CometColumnarToRow [codegen id : 10] -Input [1]: [p_promo_sk#55] +(66) CometBroadcastExchange +Input [1]: [p_promo_sk#53] +Arguments: [p_promo_sk#53] -(71) BroadcastExchange -Input [1]: [p_promo_sk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] - -(72) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_promo_sk#7] -Right keys [1]: [p_promo_sk#55] -Join type: Inner -Join condition: None +(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) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, p_promo_sk#55] +(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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.household_demographics -Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(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 -(75) CometFilter -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Condition : (isnotnull(hd_demo_sk#56) AND isnotnull(hd_income_band_sk#57)) +(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)) -(76) CometColumnarToRow [codegen id : 11] -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] +(71) CometBroadcastExchange +Input [2]: [hd_demo_sk#54, hd_income_band_sk#55] +Arguments: [hd_demo_sk#54, hd_income_band_sk#55] -(77) BroadcastExchange -Input [2]: [hd_demo_sk#56, hd_income_band_sk#57] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] - -(78) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_hdemo_sk#4] -Right keys [1]: [hd_demo_sk#56] -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 -(79) 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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57] -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#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_demo_sk#56, hd_income_band_sk#57] +(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] -(80) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#58, hd_income_band_sk#59] +(74) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#56, hd_income_band_sk#57] -(81) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_hdemo_sk#42] -Right keys [1]: [hd_demo_sk#58] -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 -(82) 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#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59] -Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_hdemo_sk#42, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_demo_sk#58, hd_income_band_sk#59] +(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] -(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer_address -Output [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] +(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 -(84) CometFilter -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Condition : isnotnull(ca_address_sk#60) +(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) -(85) CometProject -Input [5]: [ca_address_sk#60, ca_street_number#61, ca_street_name#62, ca_city#63, ca_zip#64] -Arguments: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66], [ca_address_sk#60, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#61, 10)) AS ca_street_number#65, ca_street_name#62, ca_city#63, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#64, 10)) AS ca_zip#66] - -(86) CometColumnarToRow [codegen id : 13] -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_street_number#59, 10)) AS ca_street_number#63, ca_street_name#60, ca_city#61, static_invoke(CharVarcharCodegenUtils.readSidePadding(ca_zip#62, 10)) AS ca_zip#64] -(87) BroadcastExchange -Input [5]: [ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] +(80) CometBroadcastExchange +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] -(88) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_addr_sk#5] -Right keys [1]: [ca_address_sk#60] -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 -(89) Project [codegen id : 18] -Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] -Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_address_sk#60, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66] +(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] -(90) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(91) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#43] -Right keys [1]: [ca_address_sk#67] -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 -(92) Project [codegen id : 18] -Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, c_current_addr_sk#43, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_address_sk#67, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] +(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] -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.income_band -Output [1]: [ib_income_band_sk#72] +(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 -(94) CometFilter -Input [1]: [ib_income_band_sk#72] -Condition : isnotnull(ib_income_band_sk#72) +(87) CometFilter +Input [1]: [ib_income_band_sk#70] +Condition : isnotnull(ib_income_band_sk#70) -(95) CometColumnarToRow [codegen id : 15] -Input [1]: [ib_income_band_sk#72] +(88) CometBroadcastExchange +Input [1]: [ib_income_band_sk#70] +Arguments: [ib_income_band_sk#70] -(96) BroadcastExchange -Input [1]: [ib_income_band_sk#72] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] +(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) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#57] -Right keys [1]: [ib_income_band_sk#72] -Join type: Inner -Join condition: None +(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] -(98) Project [codegen id : 18] -Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#57, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#72] +(91) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#71] -(99) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#73] +(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 -(100) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [hd_income_band_sk#59] -Right keys [1]: [ib_income_band_sk#73] -Join type: Inner -Join condition: None +(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] -(101) Project [codegen id : 18] -Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, hd_income_band_sk#59, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, ib_income_band_sk#73] - -(102) CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Output [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] +(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 -(103) CometFilter -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Condition : ((((((isnotnull(i_current_price#75) AND static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#76, 20)) IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#75 >= 64.00)) AND (i_current_price#75 <= 74.00)) AND (i_current_price#75 >= 65.00)) AND (i_current_price#75 <= 79.00)) AND isnotnull(i_item_sk#74)) - -(104) CometProject -Input [4]: [i_item_sk#74, i_current_price#75, i_color#76, i_product_name#77] -Arguments: [i_item_sk#74, i_product_name#78], [i_item_sk#74, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#77, 50)) AS i_product_name#78] +(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 static_invoke(CharVarcharCodegenUtils.readSidePadding(i_color#74, 20)) 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) CometColumnarToRow [codegen id : 17] -Input [2]: [i_item_sk#74, i_product_name#78] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(i_product_name#75, 50)) AS i_product_name#76] -(106) BroadcastExchange -Input [2]: [i_item_sk#74, i_product_name#78] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +(97) CometBroadcastExchange +Input [2]: [i_item_sk#72, i_product_name#76] +Arguments: [i_item_sk#72, i_product_name#76] -(107) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#74] -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 -(108) Project [codegen id : 18] -Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, s_store_name#37, s_zip#39, d_year#47, d_year#49, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] +(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] -(109) HashAggregate [codegen id : 18] -Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#35, d_year#47, d_year#49, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, i_item_sk#74, i_product_name#78] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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#79, sum#80, sum#81, sum#82] -Results [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -(110) HashAggregate [codegen id : 18] -Input [19]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49, count#83, sum#84, sum#85, sum#86] -Keys [15]: [i_product_name#78, i_item_sk#74, s_store_name#37, s_zip#39, ca_street_number#65, ca_street_name#62, ca_city#63, ca_zip#66, ca_street_number#68, ca_street_name#69, ca_city#70, ca_zip#71, d_year#35, d_year#47, d_year#49] +(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)#87, sum(UnscaledValue(ss_wholesale_cost#9))#88, sum(UnscaledValue(ss_list_price#10))#89, sum(UnscaledValue(ss_coupon_amt#11))#90] -Results [17]: [i_product_name#78 AS product_name#91, i_item_sk#74 AS item_sk#92, s_store_name#37 AS store_name#93, s_zip#39 AS store_zip#94, ca_street_number#65 AS b_street_number#95, ca_street_name#62 AS b_streen_name#96, ca_city#63 AS b_city#97, ca_zip#66 AS b_zip#98, ca_street_number#68 AS c_street_number#99, ca_street_name#69 AS c_street_name#100, ca_city#70 AS c_city#101, ca_zip#71 AS c_zip#102, d_year#35 AS syear#103, count(1)#87 AS cnt#104, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#88,17,2) AS s1#105, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#89,17,2) AS s2#106, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#90,17,2) AS s3#107] -(111) CometColumnarExchange -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: hashpartitioning(item_sk#92, store_name#93, store_zip#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=14] +(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) CometSort -Input [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Arguments: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107], [item_sk#92 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, store_zip#94 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] -(113) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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#119), dynamicpruningexpression(ss_sold_date_sk#119 IN dynamicpruning#120)] +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 -(114) CometFilter -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Condition : (((((((isnotnull(ss_item_sk#108) AND isnotnull(ss_ticket_number#115)) AND isnotnull(ss_store_sk#113)) AND isnotnull(ss_customer_sk#109)) AND isnotnull(ss_cdemo_sk#110)) AND isnotnull(ss_promo_sk#114)) AND isnotnull(ss_hdemo_sk#111)) AND isnotnull(ss_addr_sk#112)) +(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) CometBroadcastExchange -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] +(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] -(116) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] +(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 -(117) CometFilter -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Condition : (isnotnull(sr_item_sk#121) AND isnotnull(sr_ticket_number#122)) +(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))] -(118) CometProject -Input [3]: [sr_item_sk#121, sr_ticket_number#122, sr_returned_date_sk#123] -Arguments: [sr_item_sk#121, sr_ticket_number#122], [sr_item_sk#121, sr_ticket_number#122] +(116) CometFilter +Input [3]: [cs_item_sk#114, sale#122, refund#123] +Condition : ((isnotnull(sale#122) AND isnotnull(refund#123)) AND (cast(sale#122 as decimal(21,2)) > (2 * refund#123))) -(119) CometBroadcastHashJoin -Left output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Right output [2]: [sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_ticket_number#115], [sr_item_sk#121, sr_ticket_number#122], Inner, BuildLeft +(117) CometProject +Input [3]: [cs_item_sk#114, sale#122, refund#123] +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 [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_ticket_number#115, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, sr_item_sk#121, sr_ticket_number#122] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(121) CometExchange -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: hashpartitioning(ss_item_sk#108, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=15] - -(122) CometSort -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Arguments: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119], [ss_item_sk#108 ASC NULLS FIRST] - -(123) CometColumnarToRow [codegen id : 19] -Input [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] - -(124) ReusedExchange [Reuses operator id: 25] -Output [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(125) CometColumnarToRow [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] - -(126) HashAggregate [codegen id : 20] -Input [4]: [cs_item_sk#124, sum#125, sum#126, isEmpty#127] -Keys [1]: [cs_item_sk#124] -Functions [2]: [sum(UnscaledValue(cs_ext_list_price#128)), sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#128))#30, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31] -Results [3]: [cs_item_sk#124, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#128))#30,17,2) AS sale#132, sum(((cr_refunded_cash#129 + cr_reversed_charge#130) + cr_store_credit#131))#31 AS refund#133] - -(127) Filter [codegen id : 20] -Input [3]: [cs_item_sk#124, sale#132, refund#133] -Condition : ((isnotnull(sale#132) AND isnotnull(refund#133)) AND (cast(sale#132 as decimal(21,2)) > (2 * refund#133))) - -(128) Project [codegen id : 20] -Output [1]: [cs_item_sk#124] -Input [3]: [cs_item_sk#124, sale#132, refund#133] - -(129) Sort [codegen id : 20] -Input [1]: [cs_item_sk#124] -Arguments: [cs_item_sk#124 ASC NULLS FIRST], false, 0 - -(130) SortMergeJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [cs_item_sk#124] -Join type: Inner -Join condition: None - -(131) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119] -Input [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, cs_item_sk#124] - -(132) ReusedExchange [Reuses operator id: 193] -Output [2]: [d_date_sk#134, d_year#135] - -(133) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_sold_date_sk#119] -Right keys [1]: [d_date_sk#134] -Join type: Inner -Join condition: None - -(134) Project [codegen id : 36] -Output [11]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135] -Input [13]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, ss_sold_date_sk#119, d_date_sk#134, d_year#135] - -(135) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_store_sk#136, s_store_name#137, s_zip#138] - -(136) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_store_sk#113] -Right keys [1]: [s_store_sk#136] -Join type: Inner -Join condition: None - -(137) Project [codegen id : 36] -Output [12]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138] -Input [14]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_store_sk#113, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_sk#136, s_store_name#137, s_zip#138] - -(138) ReusedExchange [Reuses operator id: 46] -Output [6]: [c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(139) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_customer_sk#109] -Right keys [1]: [c_customer_sk#139] -Join type: Inner -Join condition: None - -(140) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] -Input [18]: [ss_item_sk#108, ss_customer_sk#109, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_customer_sk#139, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144] - -(141) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#145, d_year#146] - -(142) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_sales_date_sk#144] -Right keys [1]: [d_date_sk#145] -Join type: Inner -Join condition: None - -(143) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, c_first_sales_date_sk#144, d_date_sk#145, d_year#146] - -(144) ReusedExchange [Reuses operator id: 52] -Output [2]: [d_date_sk#147, d_year#148] - -(145) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_first_shipto_date_sk#143] -Right keys [1]: [d_date_sk#147] -Join type: Inner -Join condition: None - -(146) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, c_first_shipto_date_sk#143, d_year#146, d_date_sk#147, d_year#148] - -(147) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#149, cd_marital_status#150] - -(148) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_cdemo_sk#110] -Right keys [1]: [cd_demo_sk#149] -Join type: Inner -Join condition: None - -(149) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150] -Input [18]: [ss_item_sk#108, ss_cdemo_sk#110, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_demo_sk#149, cd_marital_status#150] - -(150) ReusedExchange [Reuses operator id: 62] -Output [2]: [cd_demo_sk#151, cd_marital_status#152] - -(151) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_cdemo_sk#140] -Right keys [1]: [cd_demo_sk#151] -Join type: Inner -Join condition: NOT (cd_marital_status#150 = cd_marital_status#152) - -(152) Project [codegen id : 36] -Output [14]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [18]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_cdemo_sk#140, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, cd_marital_status#150, cd_demo_sk#151, cd_marital_status#152] - -(153) ReusedExchange [Reuses operator id: 71] -Output [1]: [p_promo_sk#153] - -(154) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_promo_sk#114] -Right keys [1]: [p_promo_sk#153] -Join type: Inner -Join condition: None - -(155) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_promo_sk#114, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, p_promo_sk#153] - -(156) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] - -(157) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_hdemo_sk#111] -Right keys [1]: [hd_demo_sk#154] -Join type: Inner -Join condition: None - -(158) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155] -Input [15]: [ss_item_sk#108, ss_hdemo_sk#111, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_demo_sk#154, hd_income_band_sk#155] - -(159) ReusedExchange [Reuses operator id: 77] -Output [2]: [hd_demo_sk#156, hd_income_band_sk#157] - -(160) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_hdemo_sk#141] -Right keys [1]: [hd_demo_sk#156] -Join type: Inner -Join condition: None - -(161) Project [codegen id : 36] -Output [13]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157] -Input [15]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_hdemo_sk#141, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_demo_sk#156, hd_income_band_sk#157] - -(162) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(163) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_addr_sk#112] -Right keys [1]: [ca_address_sk#158] -Join type: Inner -Join condition: None - -(164) Project [codegen id : 36] -Output [16]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] -Input [18]: [ss_item_sk#108, ss_addr_sk#112, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_address_sk#158, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162] - -(165) ReusedExchange [Reuses operator id: 87] -Output [5]: [ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(166) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [c_current_addr_sk#142] -Right keys [1]: [ca_address_sk#163] -Join type: Inner -Join condition: None - -(167) Project [codegen id : 36] -Output [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [21]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, c_current_addr_sk#142, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_address_sk#163, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] - -(168) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#168] - -(169) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#155] -Right keys [1]: [ib_income_band_sk#168] -Join type: Inner -Join condition: None - -(170) Project [codegen id : 36] -Output [18]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [20]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#155, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#168] - -(171) ReusedExchange [Reuses operator id: 96] -Output [1]: [ib_income_band_sk#169] - -(172) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [hd_income_band_sk#157] -Right keys [1]: [ib_income_band_sk#169] -Join type: Inner -Join condition: None - -(173) Project [codegen id : 36] -Output [17]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, hd_income_band_sk#157, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, ib_income_band_sk#169] - -(174) ReusedExchange [Reuses operator id: 106] -Output [2]: [i_item_sk#170, i_product_name#171] - -(175) BroadcastHashJoin [codegen id : 36] -Left keys [1]: [ss_item_sk#108] -Right keys [1]: [i_item_sk#170] -Join type: Inner -Join condition: None - -(176) Project [codegen id : 36] -Output [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Input [19]: [ss_item_sk#108, ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, s_store_name#137, s_zip#138, d_year#146, d_year#148, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] - -(177) HashAggregate [codegen id : 36] -Input [18]: [ss_wholesale_cost#116, ss_list_price#117, ss_coupon_amt#118, d_year#135, d_year#146, d_year#148, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, i_item_sk#170, i_product_name#171] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#116)), partial_sum(UnscaledValue(ss_list_price#117)), partial_sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count#79, sum#172, sum#173, sum#174] -Results [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] - -(178) HashAggregate [codegen id : 36] -Input [19]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148, count#83, sum#175, sum#176, sum#177] -Keys [15]: [i_product_name#171, i_item_sk#170, s_store_name#137, s_zip#138, ca_street_number#159, ca_street_name#160, ca_city#161, ca_zip#162, ca_street_number#164, ca_street_name#165, ca_city#166, ca_zip#167, d_year#135, d_year#146, d_year#148] -Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#116)), sum(UnscaledValue(ss_list_price#117)), sum(UnscaledValue(ss_coupon_amt#118))] -Aggregate Attributes [4]: [count(1)#87, sum(UnscaledValue(ss_wholesale_cost#116))#88, sum(UnscaledValue(ss_list_price#117))#89, sum(UnscaledValue(ss_coupon_amt#118))#90] -Results [8]: [i_item_sk#170 AS item_sk#178, s_store_name#137 AS store_name#179, s_zip#138 AS store_zip#180, d_year#135 AS syear#181, count(1)#87 AS cnt#182, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#116))#88,17,2) AS s1#183, MakeDecimal(sum(UnscaledValue(ss_list_price#117))#89,17,2) AS s2#184, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#118))#90,17,2) AS s3#185] - -(179) CometColumnarExchange -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: hashpartitioning(item_sk#178, store_name#179, store_zip#180, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=16] - -(180) CometSort -Input [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185], [item_sk#178 ASC NULLS FIRST, store_name#179 ASC NULLS FIRST, store_zip#180 ASC NULLS FIRST] - -(181) CometSortMergeJoin -Left output [17]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107] -Right output [8]: [item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#179, store_zip#180], Inner, (cnt#182 <= cnt#104) - -(182) CometProject -Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] - -(183) CometExchange -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] - -(184) CometSort -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST] - -(185) CometColumnarToRow [codegen id : 37] -Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] +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#124, d_year#125] +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#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) + +(123) CometBroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +Arguments: [d_date_sk#124, d_year#125] + +(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#124, d_year#125] +Arguments: [ss_sold_date_sk#109], [d_date_sk#124], 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#124, d_year#125] +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#125], [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#125] + +(126) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] + +(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#125] +Right output [3]: [s_store_sk#126, s_store_name#127, s_zip#128] +Arguments: [ss_store_sk#103], [s_store_sk#126], 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#125, s_store_sk#126, s_store_name#127, s_zip#128] +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#125, s_store_name#127, s_zip#128], [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#125, s_store_name#127, s_zip#128] + +(129) ReusedExchange [Reuses operator id: 44] +Output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(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#125, s_store_name#127, s_zip#128] +Right output [6]: [c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Arguments: [ss_customer_sk#99], [c_customer_sk#129], 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#125, s_store_name#127, s_zip#128, c_customer_sk#129, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] + +(132) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#135, d_year#136] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134] +Right output [2]: [d_date_sk#135, d_year#136] +Arguments: [c_first_sales_date_sk#134], [d_date_sk#135], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, c_first_sales_date_sk#134, d_date_sk#135, d_year#136] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] + +(135) ReusedExchange [Reuses operator id: 49] +Output [2]: [d_date_sk#137, d_year#138] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136] +Right output [2]: [d_date_sk#137, d_year#138] +Arguments: [c_first_shipto_date_sk#133], [d_date_sk#137], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, c_first_shipto_date_sk#133, d_year#136, d_date_sk#137, d_year#138] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(138) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#139, cd_marital_status#140] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [cd_demo_sk#139, cd_marital_status#140] +Arguments: [ss_cdemo_sk#100], [cd_demo_sk#139], 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_demo_sk#139, cd_marital_status#140] +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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140], [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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] + +(141) ReusedExchange [Reuses operator id: 58] +Output [2]: [cd_demo_sk#141, cd_marital_status#142] + +(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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140] +Right output [2]: [cd_demo_sk#141, cd_marital_status#142] +Arguments: [c_current_cdemo_sk#130], [cd_demo_sk#141], Inner, NOT (cd_marital_status#140 = cd_marital_status#142), 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#125, s_store_name#127, s_zip#128, c_current_cdemo_sk#130, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, cd_marital_status#140, cd_demo_sk#141, cd_marital_status#142] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(144) ReusedExchange [Reuses operator id: 66] +Output [1]: [p_promo_sk#143] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [1]: [p_promo_sk#143] +Arguments: [ss_promo_sk#104], [p_promo_sk#143], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, p_promo_sk#143] +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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138], [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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] + +(147) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#144, hd_income_band_sk#145] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138] +Right output [2]: [hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_hdemo_sk#101], [hd_demo_sk#144], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_demo_sk#144, hd_income_band_sk#145] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] + +(150) ReusedExchange [Reuses operator id: 71] +Output [2]: [hd_demo_sk#146, hd_income_band_sk#147] + +(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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145] +Right output [2]: [hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [c_current_hdemo_sk#131], [hd_demo_sk#146], 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#125, s_store_name#127, s_zip#128, c_current_hdemo_sk#131, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_demo_sk#146, hd_income_band_sk#147] +Arguments: [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147], [ss_item_sk#98, ss_addr_sk#102, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] + +(153) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147] +Right output [5]: [ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_addr_sk#102], [ca_address_sk#148], 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#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_address_sk#148, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] + +(156) ReusedExchange [Reuses operator id: 80] +Output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(157) CometBroadcastHashJoin +Left output [16]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152] +Right output [5]: [ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [c_current_addr_sk#132], [ca_address_sk#153], Inner, BuildRight + +(158) CometProject +Input [21]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, c_current_addr_sk#132, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_address_sk#153, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(159) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#158] + +(160) CometBroadcastHashJoin +Left output [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#158] +Arguments: [hd_income_band_sk#145], [ib_income_band_sk#158], Inner, BuildRight + +(161) CometProject +Input [20]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#145, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#158] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(162) ReusedExchange [Reuses operator id: 88] +Output [1]: [ib_income_band_sk#159] + +(163) CometBroadcastHashJoin +Left output [18]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [1]: [ib_income_band_sk#159] +Arguments: [hd_income_band_sk#147], [ib_income_band_sk#159], Inner, BuildRight + +(164) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, hd_income_band_sk#147, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, ib_income_band_sk#159] +Arguments: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157], [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] + +(165) ReusedExchange [Reuses operator id: 97] +Output [2]: [i_item_sk#160, i_product_name#161] + +(166) CometBroadcastHashJoin +Left output [17]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157] +Right output [2]: [i_item_sk#160, i_product_name#161] +Arguments: [ss_item_sk#98], [i_item_sk#160], Inner, BuildRight + +(167) CometProject +Input [19]: [ss_item_sk#98, ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, s_store_name#127, s_zip#128, d_year#136, d_year#138, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Arguments: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161], [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] + +(168) CometHashAggregate +Input [18]: [ss_wholesale_cost#106, ss_list_price#107, ss_coupon_amt#108, d_year#125, d_year#136, d_year#138, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, i_item_sk#160, i_product_name#161] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138, count#77, sum#162, sum#163, sum#164] +Keys [15]: [i_product_name#161, i_item_sk#160, s_store_name#127, s_zip#128, ca_street_number#149, ca_street_name#150, ca_city#151, ca_zip#152, ca_street_number#154, ca_street_name#155, ca_city#156, ca_zip#157, d_year#125, d_year#136, d_year#138] +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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: hashpartitioning(item_sk#165, store_name#166, store_zip#167, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(171) CometSort +Input [8]: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172], [item_sk#165 ASC NULLS FIRST, store_name#166 ASC NULLS FIRST, store_zip#167 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#165, store_name#166, store_zip#167], Inner, (cnt#169 <= 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#165, store_name#166, store_zip#167, syear#168, cnt#169, s1#170, s2#171, s3#172] +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#170, s2#171, s3#172, syear#168, cnt#169], [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#170, s2#171, s3#172, syear#168, cnt#169] + +(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#170, s2#171, s3#172, syear#168, cnt#169] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 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#170, s2#171, s3#172, syear#168, cnt#169] +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#170, s2#171, s3#172, syear#168, cnt#169], [product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#169 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#170 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#170, s2#171, s3#172, syear#168, cnt#169] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 -BroadcastExchange (189) -+- * CometColumnarToRow (188) - +- CometFilter (187) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (186) +BroadcastExchange (180) ++- * CometColumnarToRow (179) + +- CometFilter (178) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (177) -(186) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#34, d_year#35] +(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 -(187) CometFilter -Input [2]: [d_date_sk#34, d_year#35] -Condition : ((isnotnull(d_year#35) AND (d_year#35 = 1999)) AND isnotnull(d_date_sk#34)) +(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)) -(188) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#34, d_year#35] +(179) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#32, d_year#33] -(189) BroadcastExchange -Input [2]: [d_date_sk#34, d_year#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] +(180) BroadcastExchange +Input [2]: [d_date_sk#32, d_year#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 113 Hosting Expression = ss_sold_date_sk#119 IN dynamicpruning#120 -BroadcastExchange (193) -+- * CometColumnarToRow (192) - +- CometFilter (191) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (190) +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) -(190) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#134, d_year#135] +(181) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#124, d_year#125] 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 -(191) CometFilter -Input [2]: [d_date_sk#134, d_year#135] -Condition : ((isnotnull(d_year#135) AND (d_year#135 = 2000)) AND isnotnull(d_date_sk#134)) +(182) CometFilter +Input [2]: [d_date_sk#124, d_year#125] +Condition : ((isnotnull(d_year#125) AND (d_year#125 = 2000)) AND isnotnull(d_date_sk#124)) -(192) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#134, d_year#135] +(183) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#124, d_year#125] -(193) BroadcastExchange -Input [2]: [d_date_sk#134, d_year#135] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] +(184) BroadcastExchange +Input [2]: [d_date_sk#124, d_year#125] +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-spark4_0/q64/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt index 41817bd434..059acab385 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/extended.txt @@ -4,278 +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 - : : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#1, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#2))#3,17,2) AS sale#4, sum(((cr_refunded_cash#5 + cr_reversed_charge#6) + cr_store_credit#7))#8 AS refund#9)] - : : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : :- Project - : : : : : : : : : : : : : : : +- SortMergeJoin - : : : : : : : : : : : : : : : :- CometColumnarToRow - : : : : : : : : : : : : : : : : +- 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 - : : : : : : : : : : : : : : : +- Sort - : : : : : : : : : : : : : : : +- Project - : : : : : : : : : : : : : : : +- Filter - : : : : : : : : : : : : : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_item_sk#10, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#11))#3,17,2) AS sale#12, sum(((cr_refunded_cash#13 + cr_reversed_charge#14) + cr_store_credit#15))#8 AS refund#16)] - : : : : : : : : : : : : : : : +- CometColumnarToRow - : : : : : : : : : : : : : : : +- 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 132 out of 242 eligible operators (54%). Final plan contains 37 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 238 out of 242 eligible operators (98%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 29748bfd47..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -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,258 +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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - 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] - InputAdapter - WholeStageCodegen (2) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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] - 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] - SortMergeJoin [ss_item_sk,cs_item_sk] - InputAdapter - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - 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] #19 - 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] #20 - 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] - 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 - WholeStageCodegen (20) - Sort [cs_item_sk] - Project [cs_item_sk] - Filter [sale,refund] - HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] - CometColumnarToRow - InputAdapter - 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-spark4_0/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt index ae9541ff99..a66acf75ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt @@ -35,9 +35,9 @@ TakeOrderedAndProject (66) : +- * Filter (31) : +- Window (30) : +- WindowGroupLimit (29) - : +- * Sort (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) : +- CometExchange (25) : +- CometHashAggregate (24) : +- CometProject (23) @@ -185,35 +185,33 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometColumnarToRow [codegen id : 1] -Input [2]: [s_state#16, sum#18] - -(27) HashAggregate [codegen id : 1] +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#16] -(28) Sort [codegen id : 1] -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(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] + +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] (29) WindowGroupLimit -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final +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#20, s_state#16] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#16, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#16], [_w0#20 DESC NULLS LAST] +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#20, s_state#16, ranking#21] -Condition : (ranking#21 <= 5) +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#20, s_state#16, ranking#21] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] (33) BroadcastExchange Input [1]: [s_state#16] @@ -226,11 +224,11 @@ Join type: LeftSemi Join condition: None (35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#22] +Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#22] +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] (37) BroadcastHashJoin [codegen id : 4] @@ -240,138 +238,138 @@ Join type: Inner Join condition: None (38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#22] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#22] +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] (39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#22] -Keys [2]: [s_state#22, s_county#8] +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#8, sum#24] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] (40) CometColumnarExchange -Input [3]: [s_state#22, s_county#8, sum#24] -Arguments: hashpartitioning(s_state#22, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +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] (41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#22, s_county#8, sum#24] +Input [3]: [s_state#21, s_county#8, sum#23] (42) HashAggregate [codegen id : 5] -Input [3]: [s_state#22, s_county#8, sum#24] -Keys [2]: [s_state#22, s_county#8] +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#8 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] +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#32, s_county#33, sum#34] +Output [3]: [s_state#31, s_county#32, sum#33] (44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#32, s_county#33, sum#34] +Input [3]: [s_state#31, s_county#32, sum#33] (45) HashAggregate [codegen id : 10] -Input [3]: [s_state#32, s_county#33, sum#34] -Keys [2]: [s_state#32, s_county#33] -Functions [1]: [sum(UnscaledValue(ss_net_profit#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#35))#25] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#35))#25,17,2) AS total_sum#36, s_state#32] +Input [3]: [s_state#31, s_county#32, sum#33] +Keys [2]: [s_state#31, s_county#32] +Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] (46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#36, s_state#32] -Keys [1]: [s_state#32] -Functions [1]: [partial_sum(total_sum#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [3]: [s_state#32, sum#39, isEmpty#40] +Input [2]: [total_sum#35, s_state#31] +Keys [1]: [s_state#31] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [3]: [s_state#31, sum#38, isEmpty#39] (47) CometColumnarExchange -Input [3]: [s_state#32, sum#39, isEmpty#40] -Arguments: hashpartitioning(s_state#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [3]: [s_state#31, sum#38, isEmpty#39] +Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#32, sum#39, isEmpty#40] +Input [3]: [s_state#31, sum#38, isEmpty#39] (49) HashAggregate [codegen id : 11] -Input [3]: [s_state#32, sum#39, isEmpty#40] -Keys [1]: [s_state#32] -Functions [1]: [sum(total_sum#36)] -Aggregate Attributes [1]: [sum(total_sum#36)#41] -Results [6]: [sum(total_sum#36)#41 AS total_sum#42, s_state#32, null AS s_county#43, 0 AS g_state#44, 1 AS g_county#45, 1 AS lochierarchy#46] +Input [3]: [s_state#31, sum#38, isEmpty#39] +Keys [1]: [s_state#31] +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#31, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] (50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#47, s_county#48, sum#49] +Output [3]: [s_state#46, s_county#47, sum#48] (51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#47, s_county#48, sum#49] +Input [3]: [s_state#46, s_county#47, sum#48] (52) HashAggregate [codegen id : 16] -Input [3]: [s_state#47, s_county#48, sum#49] -Keys [2]: [s_state#47, s_county#48] -Functions [1]: [sum(UnscaledValue(ss_net_profit#50))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#50))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#50))#25,17,2) AS total_sum#51] +Input [3]: [s_state#46, s_county#47, sum#48] +Keys [2]: [s_state#46, s_county#47] +Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] (53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#51] +Input [1]: [total_sum#50] Keys: [] -Functions [1]: [partial_sum(total_sum#51)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [2]: [sum#54, isEmpty#55] +Functions [1]: [partial_sum(total_sum#50)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [2]: [sum#53, isEmpty#54] (54) CometColumnarExchange -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] (55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] (56) HashAggregate [codegen id : 17] -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] Keys: [] -Functions [1]: [sum(total_sum#51)] -Aggregate Attributes [1]: [sum(total_sum#51)#56] -Results [6]: [sum(total_sum#51)#56 AS total_sum#57, null AS s_state#58, null AS s_county#59, 1 AS g_state#60, 1 AS g_county#61, 2 AS lochierarchy#62] +Functions [1]: [sum(total_sum#50)] +Aggregate Attributes [1]: [sum(total_sum#50)#55] +Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS s_county#58, 1 AS g_state#59, 1 AS g_county#60, 2 AS lochierarchy#61] (57) Union (58) HashAggregate [codegen id : 18] -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] +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] (59) 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=7] +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] (60) 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] +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: [] (61) CometExchange -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (62) CometSort -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] (63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] (64) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] (65) Project [codegen id : 20] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] (66) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index a679cfd1f6..191d0ef18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt @@ -40,9 +40,9 @@ TakeOrderedAndProject : +- Filter : +- Window : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -96,9 +96,9 @@ TakeOrderedAndProject : +- Filter : +- Window : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -152,9 +152,9 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -173,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 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-spark4_0/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt index 21c6dd4eb1..2832c5a348 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt @@ -59,10 +59,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index ae9541ff99..a66acf75ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -35,9 +35,9 @@ TakeOrderedAndProject (66) : +- * Filter (31) : +- Window (30) : +- WindowGroupLimit (29) - : +- * Sort (28) - : +- * HashAggregate (27) - : +- * CometColumnarToRow (26) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) : +- CometExchange (25) : +- CometHashAggregate (24) : +- CometProject (23) @@ -185,35 +185,33 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometColumnarToRow [codegen id : 1] -Input [2]: [s_state#16, sum#18] - -(27) HashAggregate [codegen id : 1] +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#11))#19] -Results [3]: [s_state#16, MakeDecimal(sum(UnscaledValue(ss_net_profit#11))#19,17,2) AS _w0#20, s_state#16] -(28) Sort [codegen id : 1] -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16 ASC NULLS FIRST, _w0#20 DESC NULLS LAST], false, 0 +(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] + +(28) CometColumnarToRow [codegen id : 1] +Input [3]: [s_state#16, _w0#19, s_state#16] (29) WindowGroupLimit -Input [3]: [s_state#16, _w0#20, s_state#16] -Arguments: [s_state#16], [_w0#20 DESC NULLS LAST], rank(_w0#20), 5, Final +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#20, s_state#16] -Arguments: [rank(_w0#20) windowspecdefinition(s_state#16, _w0#20 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#21], [s_state#16], [_w0#20 DESC NULLS LAST] +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#20, s_state#16, ranking#21] -Condition : (ranking#21 <= 5) +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#20, s_state#16, ranking#21] +Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] (33) BroadcastExchange Input [1]: [s_state#16] @@ -226,11 +224,11 @@ Join type: LeftSemi Join condition: None (35) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#22] +Output [3]: [s_store_sk#7, s_county#8, static_invoke(CharVarcharCodegenUtils.readSidePadding(s_state#9, 2)) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] (36) BroadcastExchange -Input [3]: [s_store_sk#7, s_county#8, s_state#22] +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] (37) BroadcastHashJoin [codegen id : 4] @@ -240,138 +238,138 @@ Join type: Inner Join condition: None (38) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#22] -Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#22] +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] (39) HashAggregate [codegen id : 4] -Input [3]: [ss_net_profit#2, s_county#8, s_state#22] -Keys [2]: [s_state#22, s_county#8] +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#8, sum#24] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] (40) CometColumnarExchange -Input [3]: [s_state#22, s_county#8, sum#24] -Arguments: hashpartitioning(s_state#22, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +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] (41) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#22, s_county#8, sum#24] +Input [3]: [s_state#21, s_county#8, sum#23] (42) HashAggregate [codegen id : 5] -Input [3]: [s_state#22, s_county#8, sum#24] -Keys [2]: [s_state#22, s_county#8] +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#8 AS s_county#28, 0 AS g_state#29, 0 AS g_county#30, 0 AS lochierarchy#31] +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#32, s_county#33, sum#34] +Output [3]: [s_state#31, s_county#32, sum#33] (44) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#32, s_county#33, sum#34] +Input [3]: [s_state#31, s_county#32, sum#33] (45) HashAggregate [codegen id : 10] -Input [3]: [s_state#32, s_county#33, sum#34] -Keys [2]: [s_state#32, s_county#33] -Functions [1]: [sum(UnscaledValue(ss_net_profit#35))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#35))#25] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#35))#25,17,2) AS total_sum#36, s_state#32] +Input [3]: [s_state#31, s_county#32, sum#33] +Keys [2]: [s_state#31, s_county#32] +Functions [1]: [sum(UnscaledValue(ss_net_profit#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#34))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#34))#24,17,2) AS total_sum#35, s_state#31] (46) HashAggregate [codegen id : 10] -Input [2]: [total_sum#36, s_state#32] -Keys [1]: [s_state#32] -Functions [1]: [partial_sum(total_sum#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [3]: [s_state#32, sum#39, isEmpty#40] +Input [2]: [total_sum#35, s_state#31] +Keys [1]: [s_state#31] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [3]: [s_state#31, sum#38, isEmpty#39] (47) CometColumnarExchange -Input [3]: [s_state#32, sum#39, isEmpty#40] -Arguments: hashpartitioning(s_state#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +Input [3]: [s_state#31, sum#38, isEmpty#39] +Arguments: hashpartitioning(s_state#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] (48) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#32, sum#39, isEmpty#40] +Input [3]: [s_state#31, sum#38, isEmpty#39] (49) HashAggregate [codegen id : 11] -Input [3]: [s_state#32, sum#39, isEmpty#40] -Keys [1]: [s_state#32] -Functions [1]: [sum(total_sum#36)] -Aggregate Attributes [1]: [sum(total_sum#36)#41] -Results [6]: [sum(total_sum#36)#41 AS total_sum#42, s_state#32, null AS s_county#43, 0 AS g_state#44, 1 AS g_county#45, 1 AS lochierarchy#46] +Input [3]: [s_state#31, sum#38, isEmpty#39] +Keys [1]: [s_state#31] +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#31, null AS s_county#42, 0 AS g_state#43, 1 AS g_county#44, 1 AS lochierarchy#45] (50) ReusedExchange [Reuses operator id: 40] -Output [3]: [s_state#47, s_county#48, sum#49] +Output [3]: [s_state#46, s_county#47, sum#48] (51) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#47, s_county#48, sum#49] +Input [3]: [s_state#46, s_county#47, sum#48] (52) HashAggregate [codegen id : 16] -Input [3]: [s_state#47, s_county#48, sum#49] -Keys [2]: [s_state#47, s_county#48] -Functions [1]: [sum(UnscaledValue(ss_net_profit#50))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#50))#25] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#50))#25,17,2) AS total_sum#51] +Input [3]: [s_state#46, s_county#47, sum#48] +Keys [2]: [s_state#46, s_county#47] +Functions [1]: [sum(UnscaledValue(ss_net_profit#49))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#49))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#49))#24,17,2) AS total_sum#50] (53) HashAggregate [codegen id : 16] -Input [1]: [total_sum#51] +Input [1]: [total_sum#50] Keys: [] -Functions [1]: [partial_sum(total_sum#51)] -Aggregate Attributes [2]: [sum#52, isEmpty#53] -Results [2]: [sum#54, isEmpty#55] +Functions [1]: [partial_sum(total_sum#50)] +Aggregate Attributes [2]: [sum#51, isEmpty#52] +Results [2]: [sum#53, isEmpty#54] (54) CometColumnarExchange -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] (55) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] (56) HashAggregate [codegen id : 17] -Input [2]: [sum#54, isEmpty#55] +Input [2]: [sum#53, isEmpty#54] Keys: [] -Functions [1]: [sum(total_sum#51)] -Aggregate Attributes [1]: [sum(total_sum#51)#56] -Results [6]: [sum(total_sum#51)#56 AS total_sum#57, null AS s_state#58, null AS s_county#59, 1 AS g_state#60, 1 AS g_county#61, 2 AS lochierarchy#62] +Functions [1]: [sum(total_sum#50)] +Aggregate Attributes [1]: [sum(total_sum#50)#55] +Results [6]: [sum(total_sum#50)#55 AS total_sum#56, null AS s_state#57, null AS s_county#58, 1 AS g_state#59, 1 AS g_county#60, 2 AS lochierarchy#61] (57) Union (58) HashAggregate [codegen id : 18] -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] +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] (59) 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=7] +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] (60) 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] +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: [] (61) CometExchange -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: hashpartitioning(lochierarchy#30, _w0#62, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (62) CometSort -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62], [lochierarchy#30 ASC NULLS FIRST, _w0#62 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] (63) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] (64) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#62, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#63], [lochierarchy#30, _w0#62], [total_sum#25 DESC NULLS LAST] (65) Project [codegen id : 20] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#62, rank_within_parent#63] (66) TakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#63 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#63] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt index a679cfd1f6..191d0ef18a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt @@ -40,9 +40,9 @@ TakeOrderedAndProject : +- Filter : +- Window : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -96,9 +96,9 @@ TakeOrderedAndProject : +- Filter : +- Window : +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - : +- Sort - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -152,9 +152,9 @@ TakeOrderedAndProject +- Filter +- Window +- WindowGroupLimit [COMET: WindowGroupLimit is not supported] - +- Sort - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_state#1, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#3,17,2) AS _w0#4, s_state#1)] - +- CometColumnarToRow + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate +- CometExchange +- CometHashAggregate +- CometProject @@ -173,4 +173,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 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-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index 21c6dd4eb1..2832c5a348 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -59,10 +59,10 @@ TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_count Window [_w0,s_state] WindowGroupLimit [s_state,_w0] WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter + 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] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt index 428c95b1fb..fefa1d47b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] 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 : 4] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] - -(17) HashAggregate [codegen id : 4] +(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))#15] -Results [2]: [c_customer_id#5 AS customer_id#16, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#15,17,2) AS year_total#17] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#16, year_total#17] -Condition : (isnotnull(year_total#17) AND (year_total#17 > 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#18, c_customer_id#19, c_first_name#20, c_last_name#21] +(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#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) -(21) CometProject -Input [4]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Arguments: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)) AS c_customer_id#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#24] +(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#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_customer_sk#25) +(22) CometFilter +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_customer_sk#24) -(24) CometBroadcastExchange -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -(25) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24] -Right output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_sk#18], [ss_customer_sk#25], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] +Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight -(26) CometProject -Input [7]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24, ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_year#30] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(28) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: [d_date_sk#29, d_year#30] - -(30) CometBroadcastHashJoin -Left output [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] -Right output [2]: [d_date_sk#29, d_year#30] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight - -(31) CometProject -Input [7]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] - -(32) CometHashAggregate -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#26))] - -(33) CometExchange -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] - -(35) HashAggregate [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [sum(UnscaledValue(ss_net_paid#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#26))#15] -Results [4]: [c_customer_id#22 AS customer_id#32, c_first_name#23 AS customer_first_name#33, c_last_name#24 AS customer_last_name#34, MakeDecimal(sum(UnscaledValue(ss_net_paid#26))#15,17,2) AS year_total#35] - -(36) BroadcastExchange -Input [4]: [customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#32] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] +(27) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28, d_year#29] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_year#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] + +(31) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] + +(32) CometExchange +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight + +(36) 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 -(39) CometFilter -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#36) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)))) +(37) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) -(40) CometProject -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Arguments: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42], [c_customer_sk#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)) AS c_customer_id#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#38, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#39, 30)) AS c_last_name#42] +(38) 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, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(39) 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#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_customer_sk#43) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) + +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(43) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] +Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42] -Right output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_sk#36], [ws_bill_customer_sk#43], Inner, BuildRight +(43) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -(45) CometProject -Input [7]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42, ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#46, d_year#47] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#47, d_year#48] +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] -Right output [2]: [d_date_sk#47, d_year#48] -Arguments: [ws_sold_date_sk#45], [d_date_sk#47], Inner, BuildRight +(46) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -(48) CometProject -Input [7]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45, d_date_sk#47, d_year#48] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] +(47) CometHashAggregate +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))] + +(48) CometExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#44))] - -(50) CometExchange -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] - -(52) HashAggregate [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [sum(UnscaledValue(ws_net_paid#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#44))#50] -Results [2]: [c_customer_id#40 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#44))#50,17,2) AS year_total#52] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [7]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52] -Input [8]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, customer_id#51, year_total#52] - -(57) 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] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +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))] + +(50) CometFilter +Input [2]: [customer_id#49, year_total#50] +Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#49, year_total#50] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Right output [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] +Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(55) CometFilter +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) -(59) 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#57, c_first_name#58, c_last_name#59], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#59] +(56) CometProject +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +(58) CometFilter +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#58) + +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] +Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight -(62) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(61) CometProject +Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#62, d_year#63] (63) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59] -Right output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#60], Inner, BuildRight +Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] +Right output [2]: [d_date_sk#62, d_year#63] +Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight (64) CometProject -Input [7]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59, ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#64, d_year#65] - -(66) CometBroadcastHashJoin -Left output [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] -Right output [2]: [d_date_sk#64, d_year#65] -Arguments: [ws_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(67) CometProject -Input [7]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] - -(68) CometHashAggregate -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] - -(69) CometExchange -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] - -(71) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#50] -Results [2]: [c_customer_id#57 AS customer_id#67, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#50,17,2) AS year_total#68] - -(72) BroadcastExchange -Input [2]: [customer_id#67, year_total#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#67] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#68 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END) - -(74) Project [codegen id : 4] -Output [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52, customer_id#67, year_total#68] - -(75) TakeOrderedAndProject -Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Arguments: 100, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] +Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] + +(65) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] + +(66) CometExchange +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#65, year_total#66] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] +Right output [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(71) 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#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 (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#29, d_year#30] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(81) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) +(78) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#30] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_year#29] -(83) BroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt index 966f9a40f3..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#3))#4,17,2) AS year_total#5)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#6 AS customer_id#7, c_first_name#8 AS customer_first_name#9, c_last_name#10 AS customer_last_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#12))#4,17,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue(ws_net_paid#16))#17,17,2) AS year_total#18)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#19 AS customer_id#20, MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#17,17,2) AS year_total#22)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-spark4_0/q74.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt index 6db4583da6..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74.native_iceberg_compat/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt index 428c95b1fb..fefa1d47b1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt @@ -1,79 +1,76 @@ == Physical Plan == -TakeOrderedAndProject (75) -+- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- * Project (56) - : +- * BroadcastHashJoin Inner BuildRight (55) - : :- * BroadcastHashJoin Inner BuildRight (37) - : : :- * Filter (18) - : : : +- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (36) - : : +- * HashAggregate (35) - : : +- * CometColumnarToRow (34) - : : +- CometExchange (33) - : : +- CometHashAggregate (32) - : : +- CometProject (31) - : : +- CometBroadcastHashJoin (30) - : : :- CometProject (26) - : : : +- CometBroadcastHashJoin (25) - : : : :- CometProject (21) - : : : : +- CometFilter (20) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (19) - : : : +- CometBroadcastExchange (24) - : : : +- CometFilter (23) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (22) - : : +- CometBroadcastExchange (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) - : +- BroadcastExchange (54) - : +- * Filter (53) - : +- * HashAggregate (52) - : +- * CometColumnarToRow (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometProject (40) - : : : +- CometFilter (39) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (38) - : : +- CometBroadcastExchange (43) - : : +- CometFilter (42) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (41) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * HashAggregate (71) - +- * CometColumnarToRow (70) - +- CometExchange (69) - +- CometHashAggregate (68) - +- CometProject (67) - +- CometBroadcastHashJoin (66) - :- CometProject (64) - : +- CometBroadcastHashJoin (63) - : :- CometProject (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.customer (57) - : +- CometBroadcastExchange (62) - : +- CometFilter (61) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (60) - +- ReusedExchange (65) +* 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 @@ -149,345 +146,325 @@ Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#9))] 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 : 4] -Input [5]: [c_customer_id#5, c_first_name#6, c_last_name#7, d_year#13, sum#14] - -(17) HashAggregate [codegen id : 4] +(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))#15] -Results [2]: [c_customer_id#5 AS customer_id#16, MakeDecimal(sum(UnscaledValue(ss_net_paid#9))#15,17,2) AS year_total#17] -(18) Filter [codegen id : 4] -Input [2]: [customer_id#16, year_total#17] -Condition : (isnotnull(year_total#17) AND (year_total#17 > 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#18, c_customer_id#19, c_first_name#20, c_last_name#21] +(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#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Condition : (isnotnull(c_customer_sk#18) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)))) +(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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)))) -(21) CometProject -Input [4]: [c_customer_sk#18, c_customer_id#19, c_first_name#20, c_last_name#21] -Arguments: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24], [c_customer_sk#18, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#19, 16)) AS c_customer_id#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#20, 20)) AS c_first_name#23, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#21, 30)) AS c_last_name#24] +(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#21, c_first_name#22, c_last_name#23], [c_customer_sk#17, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#18, 16)) AS c_customer_id#21, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#19, 20)) AS c_first_name#22, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#20, 30)) AS c_last_name#23] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(21) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#27), dynamicpruningexpression(ss_sold_date_sk#27 IN dynamicpruning#28)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_customer_sk)] ReadSchema: struct -(23) CometFilter -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Condition : isnotnull(ss_customer_sk#25) +(22) CometFilter +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_customer_sk#24) -(24) CometBroadcastExchange -Input [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] +(23) CometBroadcastExchange +Input [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] -(25) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24] -Right output [3]: [ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_sk#18], [ss_customer_sk#25], Inner, BuildRight +(24) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23] +Right output [3]: [ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_sk#17], [ss_customer_sk#24], Inner, BuildRight -(26) CometProject -Input [7]: [c_customer_sk#18, c_customer_id#22, c_first_name#23, c_last_name#24, ss_customer_sk#25, ss_net_paid#26, ss_sold_date_sk#27] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] +(25) CometProject +Input [7]: [c_customer_sk#17, c_customer_id#21, c_first_name#22, c_last_name#23, ss_customer_sk#24, ss_net_paid#25, ss_sold_date_sk#26] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#29, d_year#30] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(28) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) - -(29) CometBroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: [d_date_sk#29, d_year#30] - -(30) CometBroadcastHashJoin -Left output [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27] -Right output [2]: [d_date_sk#29, d_year#30] -Arguments: [ss_sold_date_sk#27], [d_date_sk#29], Inner, BuildRight - -(31) CometProject -Input [7]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, ss_sold_date_sk#27, d_date_sk#29, d_year#30] -Arguments: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30], [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] - -(32) CometHashAggregate -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, ss_net_paid#26, d_year#30] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#26))] - -(33) CometExchange -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(34) CometColumnarToRow [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] - -(35) HashAggregate [codegen id : 1] -Input [5]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30, sum#31] -Keys [4]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#30] -Functions [1]: [sum(UnscaledValue(ss_net_paid#26))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#26))#15] -Results [4]: [c_customer_id#22 AS customer_id#32, c_first_name#23 AS customer_first_name#33, c_last_name#24 AS customer_last_name#34, MakeDecimal(sum(UnscaledValue(ss_net_paid#26))#15,17,2) AS year_total#35] - -(36) BroadcastExchange -Input [4]: [customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] - -(37) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#32] -Join type: Inner -Join condition: None - -(38) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer -Output [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] +(27) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) + +(28) CometBroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: [d_date_sk#28, d_year#29] + +(29) CometBroadcastHashJoin +Left output [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_year#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(30) CometProject +Input [7]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, ss_sold_date_sk#26, d_date_sk#28, d_year#29] +Arguments: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29], [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] + +(31) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, ss_net_paid#25, d_year#29] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#25))] + +(32) CometExchange +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Arguments: hashpartitioning(c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] + +(33) CometHashAggregate +Input [5]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29, sum#30] +Keys [4]: [c_customer_id#21, c_first_name#22, c_last_name#23, d_year#29] +Functions [1]: [sum(UnscaledValue(ss_net_paid#25))] + +(34) CometBroadcastExchange +Input [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] + +(35) CometBroadcastHashJoin +Left output [2]: [customer_id#15, year_total#16] +Right output [4]: [customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Arguments: [customer_id#15], [customer_id#31], Inner, BuildRight + +(36) 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 -(39) CometFilter -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Condition : (isnotnull(c_customer_sk#36) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)))) +(37) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)))) -(40) CometProject -Input [4]: [c_customer_sk#36, c_customer_id#37, c_first_name#38, c_last_name#39] -Arguments: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42], [c_customer_sk#36, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#37, 16)) AS c_customer_id#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#38, 20)) AS c_first_name#41, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#39, 30)) AS c_last_name#42] +(38) 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, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#36, 16)) AS c_customer_id#39, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#37, 20)) AS c_first_name#40, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#38, 30)) AS c_last_name#41] -(41) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(39) 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#45), dynamicpruningexpression(ws_sold_date_sk#45 IN dynamicpruning#46)] +PartitionFilters: [isnotnull(ws_sold_date_sk#44), dynamicpruningexpression(ws_sold_date_sk#44 IN dynamicpruning#45)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(42) CometFilter -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Condition : isnotnull(ws_bill_customer_sk#43) +(40) CometFilter +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Condition : isnotnull(ws_bill_customer_sk#42) + +(41) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] -(43) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] +(42) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#35, c_customer_id#39, c_first_name#40, c_last_name#41] +Right output [3]: [ws_bill_customer_sk#42, ws_net_paid#43, ws_sold_date_sk#44] +Arguments: [c_customer_sk#35], [ws_bill_customer_sk#42], Inner, BuildRight -(44) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42] -Right output [3]: [ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_sk#36], [ws_bill_customer_sk#43], Inner, BuildRight +(43) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] -(45) CometProject -Input [7]: [c_customer_sk#36, c_customer_id#40, c_first_name#41, c_last_name#42, ws_bill_customer_sk#43, ws_net_paid#44, ws_sold_date_sk#45] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] +(44) ReusedExchange [Reuses operator id: 11] +Output [2]: [d_date_sk#46, d_year#47] -(46) ReusedExchange [Reuses operator id: 11] -Output [2]: [d_date_sk#47, d_year#48] +(45) CometBroadcastHashJoin +Left output [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, ws_sold_date_sk#44] +Right output [2]: [d_date_sk#46, d_year#47] +Arguments: [ws_sold_date_sk#44], [d_date_sk#46], Inner, BuildRight -(47) CometBroadcastHashJoin -Left output [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45] -Right output [2]: [d_date_sk#47, d_year#48] -Arguments: [ws_sold_date_sk#45], [d_date_sk#47], Inner, BuildRight +(46) CometProject +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] +Arguments: [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47], [c_customer_id#39, c_first_name#40, c_last_name#41, ws_net_paid#43, d_year#47] -(48) CometProject -Input [7]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, ws_sold_date_sk#45, d_date_sk#47, d_year#48] -Arguments: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48], [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] +(47) CometHashAggregate +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))] + +(48) CometExchange +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +Arguments: hashpartitioning(c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (49) CometHashAggregate -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, ws_net_paid#44, d_year#48] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#44))] - -(50) CometExchange -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Arguments: hashpartitioning(c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] - -(51) CometColumnarToRow [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] - -(52) HashAggregate [codegen id : 2] -Input [5]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48, sum#49] -Keys [4]: [c_customer_id#40, c_first_name#41, c_last_name#42, d_year#48] -Functions [1]: [sum(UnscaledValue(ws_net_paid#44))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#44))#50] -Results [2]: [c_customer_id#40 AS customer_id#51, MakeDecimal(sum(UnscaledValue(ws_net_paid#44))#50,17,2) AS year_total#52] - -(53) Filter [codegen id : 2] -Input [2]: [customer_id#51, year_total#52] -Condition : (isnotnull(year_total#52) AND (year_total#52 > 0.00)) - -(54) BroadcastExchange -Input [2]: [customer_id#51, year_total#52] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] - -(55) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#51] -Join type: Inner -Join condition: None - -(56) Project [codegen id : 4] -Output [7]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52] -Input [8]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, customer_id#51, year_total#52] - -(57) 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] +Input [5]: [c_customer_id#39, c_first_name#40, c_last_name#41, d_year#47, sum#48] +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))] + +(50) CometFilter +Input [2]: [customer_id#49, year_total#50] +Condition : (isnotnull(year_total#50) AND (year_total#50 > 0.00)) + +(51) CometBroadcastExchange +Input [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#49, year_total#50] + +(52) CometBroadcastHashJoin +Left output [6]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34] +Right output [2]: [customer_id#49, year_total#50] +Arguments: [customer_id#15], [customer_id#49], Inner, BuildRight + +(53) CometProject +Input [8]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, customer_id#49, year_total#50] +Arguments: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50], [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] + +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk)] ReadSchema: struct -(58) 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(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)))) +(55) CometFilter +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Condition : (isnotnull(c_customer_sk#51) AND isnotnull(static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)))) -(59) 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#57, c_first_name#58, c_last_name#59], [c_customer_sk#53, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#54, 16)) AS c_customer_id#57, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#55, 20)) AS c_first_name#58, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#56, 30)) AS c_last_name#59] +(56) CometProject +Input [4]: [c_customer_sk#51, c_customer_id#52, c_first_name#53, c_last_name#54] +Arguments: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57], [c_customer_sk#51, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_customer_id#52, 16)) AS c_customer_id#55, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_first_name#53, 20)) AS c_first_name#56, static_invoke(CharVarcharCodegenUtils.readSidePadding(c_last_name#54, 30)) AS c_last_name#57] -(60) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#62), dynamicpruningexpression(ws_sold_date_sk#62 IN dynamicpruning#63)] +PartitionFilters: [isnotnull(ws_sold_date_sk#60), dynamicpruningexpression(ws_sold_date_sk#60 IN dynamicpruning#61)] PushedFilters: [IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(61) CometFilter -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Condition : isnotnull(ws_bill_customer_sk#60) +(58) CometFilter +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Condition : isnotnull(ws_bill_customer_sk#58) + +(59) CometBroadcastExchange +Input [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] + +(60) CometBroadcastHashJoin +Left output [4]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57] +Right output [3]: [ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_sk#51], [ws_bill_customer_sk#58], Inner, BuildRight -(62) CometBroadcastExchange -Input [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] +(61) CometProject +Input [7]: [c_customer_sk#51, c_customer_id#55, c_first_name#56, c_last_name#57, ws_bill_customer_sk#58, ws_net_paid#59, ws_sold_date_sk#60] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] + +(62) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#62, d_year#63] (63) CometBroadcastHashJoin -Left output [4]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59] -Right output [3]: [ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_sk#53], [ws_bill_customer_sk#60], Inner, BuildRight +Left output [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60] +Right output [2]: [d_date_sk#62, d_year#63] +Arguments: [ws_sold_date_sk#60], [d_date_sk#62], Inner, BuildRight (64) CometProject -Input [7]: [c_customer_sk#53, c_customer_id#57, c_first_name#58, c_last_name#59, ws_bill_customer_sk#60, ws_net_paid#61, ws_sold_date_sk#62] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] - -(65) ReusedExchange [Reuses operator id: 29] -Output [2]: [d_date_sk#64, d_year#65] - -(66) CometBroadcastHashJoin -Left output [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62] -Right output [2]: [d_date_sk#64, d_year#65] -Arguments: [ws_sold_date_sk#62], [d_date_sk#64], Inner, BuildRight - -(67) CometProject -Input [7]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, ws_sold_date_sk#62, d_date_sk#64, d_year#65] -Arguments: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65], [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] - -(68) CometHashAggregate -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, ws_net_paid#61, d_year#65] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#61))] - -(69) CometExchange -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Arguments: hashpartitioning(c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(70) CometColumnarToRow [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] - -(71) HashAggregate [codegen id : 3] -Input [5]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65, sum#66] -Keys [4]: [c_customer_id#57, c_first_name#58, c_last_name#59, d_year#65] -Functions [1]: [sum(UnscaledValue(ws_net_paid#61))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#61))#50] -Results [2]: [c_customer_id#57 AS customer_id#67, MakeDecimal(sum(UnscaledValue(ws_net_paid#61))#50,17,2) AS year_total#68] - -(72) BroadcastExchange -Input [2]: [customer_id#67, year_total#68] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=7] - -(73) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [customer_id#16] -Right keys [1]: [customer_id#67] -Join type: Inner -Join condition: (CASE WHEN (year_total#52 > 0.00) THEN (year_total#68 / year_total#52) END > CASE WHEN (year_total#17 > 0.00) THEN (year_total#35 / year_total#17) END) - -(74) Project [codegen id : 4] -Output [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Input [9]: [customer_id#16, year_total#17, customer_id#32, customer_first_name#33, customer_last_name#34, year_total#35, year_total#52, customer_id#67, year_total#68] - -(75) TakeOrderedAndProject -Input [3]: [customer_id#32, customer_first_name#33, customer_last_name#34] -Arguments: 100, [customer_first_name#33 ASC NULLS FIRST, customer_id#32 ASC NULLS FIRST, customer_last_name#34 ASC NULLS FIRST], [customer_id#32, customer_first_name#33, customer_last_name#34] +Input [7]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, ws_sold_date_sk#60, d_date_sk#62, d_year#63] +Arguments: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63], [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] + +(65) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, ws_net_paid#59, d_year#63] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#59))] + +(66) CometExchange +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(67) CometHashAggregate +Input [5]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63, sum#64] +Keys [4]: [c_customer_id#55, c_first_name#56, c_last_name#57, d_year#63] +Functions [1]: [sum(UnscaledValue(ws_net_paid#59))] + +(68) CometBroadcastExchange +Input [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#65, year_total#66] + +(69) CometBroadcastHashJoin +Left output [7]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50] +Right output [2]: [customer_id#65, year_total#66] +Arguments: [customer_id#15], [customer_id#65], Inner, (CASE WHEN (year_total#50 > 0.00) THEN (year_total#66 / year_total#50) END > CASE WHEN (year_total#16 > 0.00) THEN (year_total#34 / year_total#16) END), BuildRight + +(70) CometProject +Input [9]: [customer_id#15, year_total#16, customer_id#31, customer_first_name#32, customer_last_name#33, year_total#34, year_total#50, customer_id#65, year_total#66] +Arguments: [customer_id#31, customer_first_name#32, customer_last_name#33], [customer_id#31, customer_first_name#32, customer_last_name#33] + +(71) 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#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 (79) -+- * CometColumnarToRow (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometFilter (74) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) -(76) 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 -(77) 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)) -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(79) 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=8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] -Subquery:2 Hosting operator id = 22 Hosting Expression = ss_sold_date_sk#27 IN dynamicpruning#28 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometFilter (81) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (80) +Subquery:2 Hosting operator id = 21 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#27 +BroadcastExchange (80) ++- * CometColumnarToRow (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 [2]: [d_date_sk#29, d_year#30] +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#29] 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 -(81) CometFilter -Input [2]: [d_date_sk#29, d_year#30] -Condition : (((isnotnull(d_year#30) AND (d_year#30 = 2002)) AND d_year#30 IN (2001,2002)) AND isnotnull(d_date_sk#29)) +(78) CometFilter +Input [2]: [d_date_sk#28, d_year#29] +Condition : (((isnotnull(d_year#29) AND (d_year#29 = 2002)) AND d_year#29 IN (2001,2002)) AND isnotnull(d_date_sk#28)) -(82) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#29, d_year#30] +(79) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#28, d_year#29] -(83) BroadcastExchange -Input [2]: [d_date_sk#29, d_year#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] +(80) BroadcastExchange +Input [2]: [d_date_sk#28, d_year#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] -Subquery:3 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#11 +Subquery:3 Hosting operator id = 39 Hosting Expression = ws_sold_date_sk#44 IN dynamicpruning#11 -Subquery:4 Hosting operator id = 60 Hosting Expression = ws_sold_date_sk#62 IN dynamicpruning#28 +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#60 IN dynamicpruning#27 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt index 966f9a40f3..476c7be954 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/extended.txt @@ -1,78 +1,75 @@ -TakeOrderedAndProject -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- BroadcastHashJoin - : : :- Filter - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#1 AS customer_id#2, MakeDecimal(sum(UnscaledValue(ss_net_paid#3))#4,17,2) AS year_total#5)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#6 AS customer_id#7, c_first_name#8 AS customer_first_name#9, c_last_name#10 AS customer_last_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#12))#4,17,2) AS year_total#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(c_customer_id#14 AS customer_id#15, MakeDecimal(sum(UnscaledValue(ws_net_paid#16))#17,17,2) AS year_total#18)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: Vector(c_customer_id#19 AS customer_id#20, MakeDecimal(sum(UnscaledValue(ws_net_paid#21))#17,17,2) AS year_total#22)] - +- CometColumnarToRow +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 @@ -90,4 +87,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 64 out of 85 eligible operators (75%). Final plan contains 6 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-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt index 6db4583da6..72e39422e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -1,14 +1,14 @@ -TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] - WholeStageCodegen (4) - 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 +WholeStageCodegen (1) + 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] @@ -31,72 +31,60 @@ TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] 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] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - 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 - 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 - 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] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (2) - 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 - 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 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (3) - 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 - 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 + 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 + 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-spark4_0/q75.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt index 75e4380270..ea5d8ded80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/explain.txt @@ -1,130 +1,128 @@ == Physical Plan == -* CometColumnarToRow (126) -+- CometTakeOrderedAndProject (125) - +- CometProject (124) - +- CometSortMergeJoin (123) - :- CometSort (67) - : +- CometColumnarExchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * CometColumnarToRow (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- 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) - : : +- 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 (122) - +- CometColumnarExchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (78) - : : +- CometProject (77) - : : +- CometBroadcastHashJoin (76) - : : :- CometProject (72) - : : : +- CometBroadcastHashJoin (71) - : : : :- CometFilter (69) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (70) - : : +- CometBroadcastExchange (75) - : : +- CometFilter (74) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (97) - : +- CometSortMergeJoin (96) - : :- CometSort (93) - : : +- CometExchange (92) - : : +- CometProject (91) - : : +- CometBroadcastHashJoin (90) - : : :- CometProject (88) - : : : +- CometBroadcastHashJoin (87) - : : : :- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (86) - : : +- ReusedExchange (89) - : +- CometSort (95) - : +- ReusedExchange (94) - +- CometProject (111) - +- CometSortMergeJoin (110) - :- CometSort (107) - : +- CometExchange (106) - : +- CometProject (105) - : +- CometBroadcastHashJoin (104) - : :- CometProject (102) - : : +- CometBroadcastHashJoin (101) - : : :- CometFilter (99) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (98) - : : +- ReusedExchange (100) - : +- ReusedExchange (103) - +- CometSort (109) - +- ReusedExchange (108) +* 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) + : : : +- 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 (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 @@ -411,310 +409,300 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) CometColumnarToRow [codegen id : 1] -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] - -(64) HashAggregate [codegen id : 1] +(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))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#64 AS sales_cnt#66, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#67] -(65) Filter [codegen id : 1] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Condition : isnotnull(sales_cnt#66) +(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) -(66) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(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=9] -(67) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67], [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] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] +(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 [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#68) +(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) -(70) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(71) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Right output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68], [i_item_sk#74], Inner, BuildRight +(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 -(72) CometProject -Input [10]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(74) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(75) CometBroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] -Arguments: [d_date_sk#79, d_year#80] +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(76) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Right output [2]: [d_date_sk#79, d_year#80] -Arguments: [cs_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight +(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) CometProject -Input [11]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_date_sk#79, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] +(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) CometExchange -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: hashpartitioning(cs_order_number#69, cs_item_sk#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(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, CometNativeShuffle, [plan_id=10] -(79) CometSort -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_order_number#69 ASC NULLS FIRST, cs_item_sk#68 ASC NULLS FIRST] +(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] -Output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] +(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 -Input [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84], [cr_order_number#82 ASC NULLS FIRST, cr_item_sk#81 ASC NULLS FIRST] +(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 -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Right output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cs_order_number#69, cs_item_sk#68], [cr_order_number#82, cr_item_sk#81], LeftOuter +(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 -Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86], [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] +(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#83, sales_amt#84], [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#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(84) CometFilter +Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Condition : isnotnull(ss_item_sk#85) -(86) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(85) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(87) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight -(88) CometProject -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(87) CometProject +Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(89) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#98, d_year#99] +(88) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#96, d_year#97] -(90) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Right output [2]: [d_date_sk#98, d_year#99] -Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Right output [2]: [d_date_sk#96, d_year#97] +Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight -(91) CometProject -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(90) CometProject +Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -(92) CometExchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(91) CometExchange +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(93) CometSort -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST] +(92) CometSort +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] -(94) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(93) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -(95) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +(94) CometSort +Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] -(96) CometSortMergeJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Right output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [ss_ticket_number#88, ss_item_sk#87], [sr_ticket_number#101, sr_item_sk#100], LeftOuter +(95) CometSortMergeJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter -(97) CometProject -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105], [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#104, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#105] +(96) CometProject +Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#110), dynamicpruningexpression(ws_sold_date_sk#110 IN dynamicpruning#111)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter -Input [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Condition : isnotnull(ws_item_sk#106) +(98) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(100) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(99) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(101) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Right output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106], [i_item_sk#112], Inner, BuildRight +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(102) CometProject -Input [10]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(101) CometProject +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(103) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#117, d_year#118] +(102) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#115, d_year#116] -(104) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Right output [2]: [d_date_sk#117, d_year#118] -Arguments: [ws_sold_date_sk#110], [d_date_sk#117], Inner, BuildRight +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(105) CometProject -Input [11]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_date_sk#117, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] +(104) CometProject +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(106) CometExchange -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: hashpartitioning(ws_order_number#107, ws_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(105) CometExchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(107) CometSort -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_order_number#107 ASC NULLS FIRST, ws_item_sk#106 ASC NULLS FIRST] +(106) CometSort +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] -(108) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] +(107) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(109) CometSort -Input [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122], [wr_order_number#120 ASC NULLS FIRST, wr_item_sk#119 ASC NULLS FIRST] +(108) CometSort +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] -(110) CometSortMergeJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Right output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [ws_order_number#107, ws_item_sk#106], [wr_order_number#120, wr_item_sk#119], LeftOuter +(109) CometSortMergeJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter -(111) CometProject -Input [13]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118, wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124], [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, (ws_quantity#108 - coalesce(wr_return_quantity#121, 0)) AS sales_cnt#123, (ws_ext_sales_price#109 - coalesce(wr_return_amt#122, 0.00)) AS sales_amt#124] +(110) CometProject +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] -(112) CometUnion -Child 0 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Child 1 Input [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105] -Child 2 Input [7]: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124] +(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#83, sales_amt#84] +Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] +Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] -(113) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(114) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(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#83, sales_amt#84] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(116) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] - -(117) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +(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#83, sales_amt#84] +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#83), partial_sum(UnscaledValue(sales_amt#84))] -(118) CometColumnarToRow [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] +(116) CometExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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] -(119) HashAggregate [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] -Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] +(117) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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#83), sum(UnscaledValue(sales_amt#84))] -(120) Filter [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Condition : isnotnull(sales_cnt#127) +(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#125, sales_amt#126] +Condition : isnotnull(sales_cnt#125) -(121) CometColumnarExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(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#125, sales_amt#126] +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] -(122) CometSort -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128], [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST] +(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#125, sales_amt#126] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [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] -(123) 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#66, sales_amt#67] -Right output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], Inner, ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) +(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#125, sales_amt#126] +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#125 as decimal(17,2))) < 0.90000000000000000000) -(124) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +(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#125, sales_amt#126] +Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] -(125) CometTakeOrderedAndProject -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#133 ASC NULLS FIRST,sales_amt_diff#134 ASC NULLS FIRST], output=[prev_year#129,year#130,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#131,curr_yr_cnt#132,sales_cnt_diff#133,sales_amt_diff#134]), [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], 100, 0, [sales_cnt_diff#133 ASC NULLS FIRST, sales_amt_diff#134 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(123) CometTakeOrderedAndProject +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST,sales_amt_diff#132 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST, sales_amt_diff#132 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -(126) CometColumnarToRow [codegen id : 3] -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(124) CometColumnarToRow [codegen id : 1] +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (130) -+- * CometColumnarToRow (129) - +- CometFilter (128) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (127) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(127) 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 -(128) 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)) -(129) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(130) 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=16] @@ -722,33 +710,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometFilter (132) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (131) +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) -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(132) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(133) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#80] +(131) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(134) BroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] +(132) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] 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#91 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#110 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt index 42b42735de..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/extended.txt @@ -3,172 +3,170 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2, i_class_id#3, i_category_id#4, i_manufact_id#5, sum(sales_cnt#6)#7 AS sales_cnt#8, MakeDecimal(sum(UnscaledValue(sales_amt#9))#10,18,2) AS sales_amt#11)] - : +- CometColumnarToRow - : +- CometExchange + : +- CometExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- 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 + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +- CometSort - +- CometColumnarExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#12, i_brand_id#13, i_class_id#14, i_category_id#15, i_manufact_id#16, sum(sales_cnt#17)#7 AS sales_cnt#18, MakeDecimal(sum(UnscaledValue(sales_amt#19))#10,18,2) AS sales_amt#20)] - +- CometColumnarToRow - +- CometExchange + +- CometExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- 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 - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- 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 + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometProject + +- CometSortMergeJoin + :- CometSort + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 155 out of 167 eligible operators (92%). Final plan contains 5 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-spark4_0/q75.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt index 9a3700b0b1..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75.native_iceberg_compat/simplified.txt @@ -1,150 +1,144 @@ -WholeStageCodegen (3) +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] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [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] CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (1) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - 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] #3 - 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] #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] #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] - 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] #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] - 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] #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] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + 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] #2 + 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] #3 + 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] #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] #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] + 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] #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] + 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] #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] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (2) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - 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] #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] - 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] #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] - 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] #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] - 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] #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] #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] #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] + 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] #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] + 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] #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] + 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] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt index 75e4380270..ea5d8ded80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt @@ -1,130 +1,128 @@ == Physical Plan == -* CometColumnarToRow (126) -+- CometTakeOrderedAndProject (125) - +- CometProject (124) - +- CometSortMergeJoin (123) - :- CometSort (67) - : +- CometColumnarExchange (66) - : +- * Filter (65) - : +- * HashAggregate (64) - : +- * CometColumnarToRow (63) - : +- CometExchange (62) - : +- CometHashAggregate (61) - : +- CometHashAggregate (60) - : +- CometExchange (59) - : +- CometHashAggregate (58) - : +- CometUnion (57) - : :- 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) - : : +- 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 (122) - +- CometColumnarExchange (121) - +- * Filter (120) - +- * HashAggregate (119) - +- * CometColumnarToRow (118) - +- CometExchange (117) - +- CometHashAggregate (116) - +- CometHashAggregate (115) - +- CometExchange (114) - +- CometHashAggregate (113) - +- CometUnion (112) - :- CometProject (83) - : +- CometSortMergeJoin (82) - : :- CometSort (79) - : : +- CometExchange (78) - : : +- CometProject (77) - : : +- CometBroadcastHashJoin (76) - : : :- CometProject (72) - : : : +- CometBroadcastHashJoin (71) - : : : :- CometFilter (69) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (68) - : : : +- ReusedExchange (70) - : : +- CometBroadcastExchange (75) - : : +- CometFilter (74) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (73) - : +- CometSort (81) - : +- ReusedExchange (80) - :- CometProject (97) - : +- CometSortMergeJoin (96) - : :- CometSort (93) - : : +- CometExchange (92) - : : +- CometProject (91) - : : +- CometBroadcastHashJoin (90) - : : :- CometProject (88) - : : : +- CometBroadcastHashJoin (87) - : : : :- CometFilter (85) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (84) - : : : +- ReusedExchange (86) - : : +- ReusedExchange (89) - : +- CometSort (95) - : +- ReusedExchange (94) - +- CometProject (111) - +- CometSortMergeJoin (110) - :- CometSort (107) - : +- CometExchange (106) - : +- CometProject (105) - : +- CometBroadcastHashJoin (104) - : :- CometProject (102) - : : +- CometBroadcastHashJoin (101) - : : :- CometFilter (99) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (98) - : : +- ReusedExchange (100) - : +- ReusedExchange (103) - +- CometSort (109) - +- ReusedExchange (108) +* 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) + : : : +- 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 (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 @@ -411,310 +409,300 @@ Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#2 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] -(63) CometColumnarToRow [codegen id : 1] -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] - -(64) HashAggregate [codegen id : 1] +(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))] -Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65] -Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#64 AS sales_cnt#66, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#67] -(65) Filter [codegen id : 1] -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Condition : isnotnull(sales_cnt#66) +(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) -(66) CometColumnarExchange -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] +(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=9] -(67) CometSort -Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67] -Arguments: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67], [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] +(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] -(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] +(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 [] -PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_sold_date_sk#72 IN dynamicpruning#73)] +PartitionFilters: [isnotnull(cs_sold_date_sk#70), dynamicpruningexpression(cs_sold_date_sk#70 IN dynamicpruning#71)] PushedFilters: [IsNotNull(cs_item_sk)] ReadSchema: struct -(69) CometFilter -Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Condition : isnotnull(cs_item_sk#68) +(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) -(70) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(71) CometBroadcastHashJoin -Left output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72] -Right output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68], [i_item_sk#74], Inner, BuildRight +(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 -(72) CometProject -Input [10]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] +(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] -(73) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(74) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(75) CometBroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] -Arguments: [d_date_sk#79, d_year#80] +(74) CometBroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] +Arguments: [d_date_sk#77, d_year#78] -(76) CometBroadcastHashJoin -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Right output [2]: [d_date_sk#79, d_year#80] -Arguments: [cs_sold_date_sk#72], [d_date_sk#79], Inner, BuildRight +(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) CometProject -Input [11]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_date_sk#79, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] +(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) CometExchange -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: hashpartitioning(cs_order_number#69, cs_item_sk#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] +(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, CometNativeShuffle, [plan_id=10] -(79) CometSort -Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Arguments: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80], [cs_order_number#69 ASC NULLS FIRST, cs_item_sk#68 ASC NULLS FIRST] +(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] -Output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] +(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 -Input [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84], [cr_order_number#82 ASC NULLS FIRST, cr_item_sk#81 ASC NULLS FIRST] +(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 -Left output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80] -Right output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [cs_order_number#69, cs_item_sk#68], [cr_order_number#82, cr_item_sk#81], LeftOuter +(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 -Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86], [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#85, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#86] +(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#83, sales_amt#84], [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#83, (cs_ext_sales_price#69 - coalesce(cr_return_amount#82, 0.00)) AS sales_amt#84] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] +(83) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#91), dynamicpruningexpression(ss_sold_date_sk#91 IN dynamicpruning#92)] +PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_sold_date_sk#89 IN dynamicpruning#90)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(85) CometFilter -Input [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Condition : isnotnull(ss_item_sk#87) +(84) CometFilter +Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Condition : isnotnull(ss_item_sk#85) -(86) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(85) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(87) CometBroadcastHashJoin -Left output [5]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91] -Right output [5]: [i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87], [i_item_sk#93], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89] +Right output [5]: [i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85], [i_item_sk#91], Inner, BuildRight -(88) CometProject -Input [10]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_item_sk#93, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] +(87) CometProject +Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#91, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] -(89) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#98, d_year#99] +(88) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#96, d_year#97] -(90) CometBroadcastHashJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97] -Right output [2]: [d_date_sk#98, d_year#99] -Arguments: [ss_sold_date_sk#91], [d_date_sk#98], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95] +Right output [2]: [d_date_sk#96, d_year#97] +Arguments: [ss_sold_date_sk#89], [d_date_sk#96], Inner, BuildRight -(91) CometProject -Input [11]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, ss_sold_date_sk#91, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_date_sk#98, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] +(90) CometProject +Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_date_sk#96, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] -(92) CometExchange -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: hashpartitioning(ss_ticket_number#88, ss_item_sk#87, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] +(91) CometExchange +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] -(93) CometSort -Input [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Arguments: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99], [ss_ticket_number#88 ASC NULLS FIRST, ss_item_sk#87 ASC NULLS FIRST] +(92) CometSort +Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Arguments: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97], [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST] -(94) ReusedExchange [Reuses operator id: 36] -Output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] +(93) ReusedExchange [Reuses operator id: 36] +Output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] -(95) CometSort -Input [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103], [sr_ticket_number#101 ASC NULLS FIRST, sr_item_sk#100 ASC NULLS FIRST] +(94) CometSort +Input [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101], [sr_ticket_number#99 ASC NULLS FIRST, sr_item_sk#98 ASC NULLS FIRST] -(96) CometSortMergeJoin -Left output [9]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99] -Right output [4]: [sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [ss_ticket_number#88, ss_item_sk#87], [sr_ticket_number#101, sr_item_sk#100], LeftOuter +(95) CometSortMergeJoin +Left output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97] +Right output [4]: [sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [ss_ticket_number#86, ss_item_sk#85], [sr_ticket_number#99, sr_item_sk#98], LeftOuter -(97) CometProject -Input [13]: [ss_item_sk#87, ss_ticket_number#88, ss_quantity#89, ss_ext_sales_price#90, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, d_year#99, sr_item_sk#100, sr_ticket_number#101, sr_return_quantity#102, sr_return_amt#103] -Arguments: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105], [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, (ss_quantity#89 - coalesce(sr_return_quantity#102, 0)) AS sales_cnt#104, (ss_ext_sales_price#90 - coalesce(sr_return_amt#103, 0.00)) AS sales_amt#105] +(96) CometProject +Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, d_year#97, sr_item_sk#98, sr_ticket_number#99, sr_return_quantity#100, sr_return_amt#101] +Arguments: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103], [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, (ss_quantity#87 - coalesce(sr_return_quantity#100, 0)) AS sales_cnt#102, (ss_ext_sales_price#88 - coalesce(sr_return_amt#101, 0.00)) AS sales_amt#103] -(98) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] +(97) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#110), dynamicpruningexpression(ws_sold_date_sk#110 IN dynamicpruning#111)] +PartitionFilters: [isnotnull(ws_sold_date_sk#108), dynamicpruningexpression(ws_sold_date_sk#108 IN dynamicpruning#109)] PushedFilters: [IsNotNull(ws_item_sk)] ReadSchema: struct -(99) CometFilter -Input [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Condition : isnotnull(ws_item_sk#106) +(98) CometFilter +Input [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Condition : isnotnull(ws_item_sk#104) -(100) ReusedExchange [Reuses operator id: 6] -Output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(99) ReusedExchange [Reuses operator id: 6] +Output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(101) CometBroadcastHashJoin -Left output [5]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110] -Right output [5]: [i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106], [i_item_sk#112], Inner, BuildRight +(100) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108] +Right output [5]: [i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104], [i_item_sk#110], Inner, BuildRight -(102) CometProject -Input [10]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_item_sk#112, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] +(101) CometProject +Input [10]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_item_sk#110, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] -(103) ReusedExchange [Reuses operator id: 75] -Output [2]: [d_date_sk#117, d_year#118] +(102) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#115, d_year#116] -(104) CometBroadcastHashJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116] -Right output [2]: [d_date_sk#117, d_year#118] -Arguments: [ws_sold_date_sk#110], [d_date_sk#117], Inner, BuildRight +(103) CometBroadcastHashJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114] +Right output [2]: [d_date_sk#115, d_year#116] +Arguments: [ws_sold_date_sk#108], [d_date_sk#115], Inner, BuildRight -(105) CometProject -Input [11]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, ws_sold_date_sk#110, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_date_sk#117, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] +(104) CometProject +Input [11]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, ws_sold_date_sk#108, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_date_sk#115, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] -(106) CometExchange -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: hashpartitioning(ws_order_number#107, ws_item_sk#106, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] +(105) CometExchange +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: hashpartitioning(ws_order_number#105, ws_item_sk#104, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(107) CometSort -Input [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Arguments: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118], [ws_order_number#107 ASC NULLS FIRST, ws_item_sk#106 ASC NULLS FIRST] +(106) CometSort +Input [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Arguments: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116], [ws_order_number#105 ASC NULLS FIRST, ws_item_sk#104 ASC NULLS FIRST] -(108) ReusedExchange [Reuses operator id: 53] -Output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] +(107) ReusedExchange [Reuses operator id: 53] +Output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] -(109) CometSort -Input [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122], [wr_order_number#120 ASC NULLS FIRST, wr_item_sk#119 ASC NULLS FIRST] +(108) CometSort +Input [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120], [wr_order_number#118 ASC NULLS FIRST, wr_item_sk#117 ASC NULLS FIRST] -(110) CometSortMergeJoin -Left output [9]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118] -Right output [4]: [wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [ws_order_number#107, ws_item_sk#106], [wr_order_number#120, wr_item_sk#119], LeftOuter +(109) CometSortMergeJoin +Left output [9]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116] +Right output [4]: [wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [ws_order_number#105, ws_item_sk#104], [wr_order_number#118, wr_item_sk#117], LeftOuter -(111) CometProject -Input [13]: [ws_item_sk#106, ws_order_number#107, ws_quantity#108, ws_ext_sales_price#109, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, d_year#118, wr_item_sk#119, wr_order_number#120, wr_return_quantity#121, wr_return_amt#122] -Arguments: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124], [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, (ws_quantity#108 - coalesce(wr_return_quantity#121, 0)) AS sales_cnt#123, (ws_ext_sales_price#109 - coalesce(wr_return_amt#122, 0.00)) AS sales_amt#124] +(110) CometProject +Input [13]: [ws_item_sk#104, ws_order_number#105, ws_quantity#106, ws_ext_sales_price#107, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, d_year#116, wr_item_sk#117, wr_order_number#118, wr_return_quantity#119, wr_return_amt#120] +Arguments: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122], [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, (ws_quantity#106 - coalesce(wr_return_quantity#119, 0)) AS sales_cnt#121, (ws_ext_sales_price#107 - coalesce(wr_return_amt#120, 0.00)) AS sales_amt#122] -(112) CometUnion -Child 0 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Child 1 Input [7]: [d_year#99, i_brand_id#94, i_class_id#95, i_category_id#96, i_manufact_id#97, sales_cnt#104, sales_amt#105] -Child 2 Input [7]: [d_year#118, i_brand_id#113, i_class_id#114, i_category_id#115, i_manufact_id#116, sales_cnt#123, sales_amt#124] +(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#83, sales_amt#84] +Child 1 Input [7]: [d_year#97, i_brand_id#92, i_class_id#93, i_category_id#94, i_manufact_id#95, sales_cnt#102, sales_amt#103] +Child 2 Input [7]: [d_year#116, i_brand_id#111, i_class_id#112, i_category_id#113, i_manufact_id#114, sales_cnt#121, sales_amt#122] -(113) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(114) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] +(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#83, sales_amt#84] +Arguments: hashpartitioning(d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(115) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] +(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#83, sales_amt#84] +Keys [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#83, sales_amt#84] Functions: [] -(116) CometHashAggregate -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#85, sales_amt#86] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [partial_sum(sales_cnt#85), partial_sum(UnscaledValue(sales_amt#86))] - -(117) CometExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=14] +(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#83, sales_amt#84] +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#83), partial_sum(UnscaledValue(sales_amt#84))] -(118) CometColumnarToRow [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] +(116) CometExchange +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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] -(119) HashAggregate [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#125, sum#126] -Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78] -Functions [2]: [sum(sales_cnt#85), sum(UnscaledValue(sales_amt#86))] -Aggregate Attributes [2]: [sum(sales_cnt#85)#64, sum(UnscaledValue(sales_amt#86))#65] -Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#85)#64 AS sales_cnt#127, MakeDecimal(sum(UnscaledValue(sales_amt#86))#65,18,2) AS sales_amt#128] +(117) CometHashAggregate +Input [7]: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sum#123, sum#124] +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#83), sum(UnscaledValue(sales_amt#84))] -(120) Filter [codegen id : 2] -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Condition : isnotnull(sales_cnt#127) +(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#125, sales_amt#126] +Condition : isnotnull(sales_cnt#125) -(121) CometColumnarExchange -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=15] +(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#125, sales_amt#126] +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] -(122) CometSort -Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128], [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST] +(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#125, sales_amt#126] +Arguments: [d_year#78, i_brand_id#73, i_class_id#74, i_category_id#75, i_manufact_id#76, sales_cnt#125, sales_amt#126], [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] -(123) 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#66, sales_amt#67] -Right output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78], Inner, ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#127 as decimal(17,2))) < 0.90000000000000000000) +(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#125, sales_amt#126] +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#125 as decimal(17,2))) < 0.90000000000000000000) -(124) CometProject -Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#127, sales_amt#128] -Arguments: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], [d_year#80 AS prev_year#129, d_year#14 AS year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#127 AS prev_yr_cnt#131, sales_cnt#66 AS curr_yr_cnt#132, (sales_cnt#66 - sales_cnt#127) AS sales_cnt_diff#133, (sales_amt#67 - sales_amt#128) AS sales_amt_diff#134] +(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#125, sales_amt#126] +Arguments: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], [d_year#78 AS prev_year#127, d_year#14 AS year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#125 AS prev_yr_cnt#129, sales_cnt#64 AS curr_yr_cnt#130, (sales_cnt#64 - sales_cnt#125) AS sales_cnt_diff#131, (sales_amt#65 - sales_amt#126) AS sales_amt_diff#132] -(125) CometTakeOrderedAndProject -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#133 ASC NULLS FIRST,sales_amt_diff#134 ASC NULLS FIRST], output=[prev_year#129,year#130,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#131,curr_yr_cnt#132,sales_cnt_diff#133,sales_amt_diff#134]), [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134], 100, 0, [sales_cnt_diff#133 ASC NULLS FIRST, sales_amt_diff#134 ASC NULLS FIRST], [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(123) CometTakeOrderedAndProject +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[sales_cnt_diff#131 ASC NULLS FIRST,sales_amt_diff#132 ASC NULLS FIRST], output=[prev_year#127,year#128,i_brand_id#8,i_class_id#9,i_category_id#10,i_manufact_id#12,prev_yr_cnt#129,curr_yr_cnt#130,sales_cnt_diff#131,sales_amt_diff#132]), [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132], 100, 0, [sales_cnt_diff#131 ASC NULLS FIRST, sales_amt_diff#132 ASC NULLS FIRST], [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] -(126) CometColumnarToRow [codegen id : 3] -Input [10]: [prev_year#129, year#130, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#131, curr_yr_cnt#132, sales_cnt_diff#133, sales_amt_diff#134] +(124) CometColumnarToRow [codegen id : 1] +Input [10]: [prev_year#127, year#128, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#129, curr_yr_cnt#130, sales_cnt_diff#131, sales_amt_diff#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 -BroadcastExchange (130) -+- * CometColumnarToRow (129) - +- CometFilter (128) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (127) +BroadcastExchange (128) ++- * CometColumnarToRow (127) + +- CometFilter (126) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) -(127) 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 -(128) 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)) -(129) CometColumnarToRow [codegen id : 1] +(127) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#13, d_year#14] -(130) 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=16] @@ -722,33 +710,33 @@ Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#26 IN d Subquery:3 Hosting operator id = 40 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 -Subquery:4 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73 -BroadcastExchange (134) -+- * CometColumnarToRow (133) - +- CometFilter (132) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (131) +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) -(131) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#79, d_year#80] +(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 -(132) CometFilter -Input [2]: [d_date_sk#79, d_year#80] -Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79)) +(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)) -(133) CometColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#79, d_year#80] +(131) CometColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#77, d_year#78] -(134) BroadcastExchange -Input [2]: [d_date_sk#79, d_year#80] +(132) BroadcastExchange +Input [2]: [d_date_sk#77, d_year#78] 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#91 IN dynamicpruning#73 +Subquery:5 Hosting operator id = 83 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#71 -Subquery:6 Hosting operator id = 98 Hosting Expression = ws_sold_date_sk#110 IN dynamicpruning#73 +Subquery:6 Hosting operator id = 97 Hosting Expression = ws_sold_date_sk#108 IN dynamicpruning#71 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt index 42b42735de..22ccc17641 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/extended.txt @@ -3,172 +3,170 @@ CometColumnarToRow +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1, i_brand_id#2, i_class_id#3, i_category_id#4, i_manufact_id#5, sum(sales_cnt#6)#7 AS sales_cnt#8, MakeDecimal(sum(UnscaledValue(sales_amt#9))#10,18,2) AS sales_amt#11)] - : +- CometColumnarToRow - : +- CometExchange + : +- CometExchange + : +- CometFilter + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometUnion - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : : +- SubqueryBroadcast - : : : : : +- BroadcastExchange - : : : : : +- 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 - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometExchange - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : : +- ReusedSubquery - : : : : +- CometBroadcastExchange - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - : +- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometExchange + : +- CometHashAggregate + : +- CometUnion + : :- 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 + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometExchange + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : : +- ReusedSubquery + : : : : +- CometBroadcastExchange + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + : +- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +- CometSort - +- CometColumnarExchange - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#12, i_brand_id#13, i_class_id#14, i_category_id#15, i_manufact_id#16, sum(sales_cnt#17)#7 AS sales_cnt#18, MakeDecimal(sum(UnscaledValue(sales_amt#19))#10,18,2) AS sales_amt#20)] - +- CometColumnarToRow - +- CometExchange + +- CometExchange + +- CometFilter + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- 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 - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - :- CometProject - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- ReusedSubquery - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometProject - +- CometSortMergeJoin - :- CometSort - : +- CometExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometSort - +- CometExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- 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 + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + :- CometProject + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- ReusedSubquery + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometProject + +- CometSortMergeJoin + :- CometSort + : +- CometExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometSort + +- CometExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Comet accelerated 155 out of 167 eligible operators (92%). Final plan contains 5 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-spark4_0/q75/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt index 9a3700b0b1..d0520c8b63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -1,150 +1,144 @@ -WholeStageCodegen (3) +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] CometProject [d_year,d_year,sales_cnt,sales_cnt,sales_amt,sales_amt] [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] CometSortMergeJoin [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] CometSort [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 - WholeStageCodegen (1) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 - 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] #3 - 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] #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] #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] - 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] #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] - 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] #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] + CometExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + 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] #2 + 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] #3 + 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] #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] #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] + 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] #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] + 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] #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] - CometColumnarExchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 - WholeStageCodegen (2) - Filter [sales_cnt] - HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] - CometColumnarToRow - InputAdapter - 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] #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] - 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] #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] - 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] #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] - 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] #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] #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] #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] + 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] #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] + 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] #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] + 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] #12 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt index e1a205bd43..9554c6732e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/explain.txt @@ -1,109 +1,107 @@ == Physical Plan == -* CometColumnarToRow (105) -+- CometTakeOrderedAndProject (104) - +- CometHashAggregate (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * HashAggregate (85) - : +- * CometColumnarToRow (84) - : +- CometColumnarExchange (83) - : +- * HashAggregate (82) - : +- Union (81) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (18) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (51) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) - : : :- BroadcastExchange (41) - : : : +- * HashAggregate (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometExchange (38) - : : : +- CometHashAggregate (37) - : : : +- CometProject (36) - : : : +- CometBroadcastHashJoin (35) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (34) - : : +- * HashAggregate (49) - : : +- * CometColumnarToRow (48) - : : +- CometExchange (47) - : : +- CometHashAggregate (46) - : : +- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (42) - : : +- ReusedExchange (43) - : +- * Project (80) - : +- * BroadcastHashJoin LeftOuter BuildRight (79) - : :- * HashAggregate (65) - : : +- * CometColumnarToRow (64) - : : +- CometExchange (63) - : : +- CometHashAggregate (62) - : : +- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometFilter (53) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (57) - : +- BroadcastExchange (78) - : +- * HashAggregate (77) - : +- * CometColumnarToRow (76) - : +- CometExchange (75) - : +- CometHashAggregate (74) - : +- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometFilter (67) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (66) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (92) - : +- * CometColumnarToRow (91) - : +- CometColumnarExchange (90) - : +- * HashAggregate (89) - : +- * HashAggregate (88) - : +- * CometColumnarToRow (87) - : +- ReusedExchange (86) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometColumnarExchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- * CometColumnarToRow (94) - +- ReusedExchange (93) +* 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) + : +- * 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 @@ -179,465 +177,445 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Input [3]: [s_store_sk#8, sum#9, sum#10] Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#8, sum#9, sum#10] - -(17) HashAggregate [codegen id : 2] +(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))#11, sum(UnscaledValue(ss_net_profit#3))#12] -Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#15) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(21) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(22) CometProject -Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] -Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(24) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#21] -Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight - -(25) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] - -(26) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] - -(27) CometExchange -Input [3]: [s_store_sk#21, sum#22, sum#23] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#26, profit_loss#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None +(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] + +(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) 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) 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] + +(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))] -(32) Project [codegen id : 2] -Output [5]: [store channel AS channel#28, s_store_sk#8 AS id#29, sales#13, coalesce(returns#26, 0.00) AS returns#30, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#31] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +(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 + +(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] + +(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#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] -Right output [1]: [d_date_sk#37] -Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight +(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) CometProject -Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] -Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +(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] -(37) CometHashAggregate -Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] +(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))] -(38) CometExchange -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(39) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(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))] -(40) HashAggregate [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] -Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(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#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(44) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] -Right output [1]: [d_date_sk#48] -Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight +(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) CometProject -Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] -Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] +(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) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(47) CometExchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometColumnarToRow -Input [2]: [sum#49, sum#50] - -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] + +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] -(50) BroadcastNestedLoopJoin [codegen id : 4] +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(51) Project [codegen id : 4] -Output [5]: [catalog channel AS channel#55, cs_call_center_sk#32 AS id#56, sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#57] -Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] +(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] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_web_page_sk#58) +(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) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(55) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight +(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 -(56) CometProject -Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] -Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +(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] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(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 -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(60) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -Right output [1]: [wp_web_page_sk#64] -Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(61) CometProject -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +(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 -(62) CometHashAggregate -Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] +(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] -(63) CometExchange -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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))] -(64) CometColumnarToRow [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(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) HashAggregate [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] -Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] +(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))] -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +(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#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Condition : isnotnull(wr_web_page_sk#71) +(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) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(69) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Right output [1]: [d_date_sk#76] -Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner, BuildRight +(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 -(70) CometProject -Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] -Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +(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] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(72) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] -Right output [1]: [wp_web_page_sk#77] -Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner, BuildRight +(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 -(73) CometProject -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +(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] -(74) CometHashAggregate -Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] - -(75) CometExchange -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] - -(77) HashAggregate [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] - -(78) BroadcastExchange -Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(79) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wp_web_page_sk#64] -Right keys [1]: [wp_web_page_sk#77] -Join type: LeftOuter -Join condition: None +(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] -(80) Project [codegen id : 6] -Output [5]: [web channel AS channel#84, wp_web_page_sk#64 AS id#85, sales#69, coalesce(returns#82, 0.00) AS returns#86, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#87] -Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] - -(81) Union - -(82) HashAggregate [codegen id : 7] -Input [5]: [channel#28, id#29, sales#13, returns#30, profit#31] -Keys [2]: [channel#28, id#29] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#30), partial_sum(profit#31)] -Aggregate Attributes [6]: [sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Results [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] - -(83) CometColumnarExchange -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#28, id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(84) CometColumnarToRow [codegen id : 8] -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] - -(85) HashAggregate [codegen id : 8] -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [2]: [channel#28, id#29] -Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] -Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] -Results [5]: [channel#28, id#29, cast(sum(sales#13)#100 as decimal(37,2)) AS sales#103, cast(sum(returns#30)#101 as decimal(37,2)) AS returns#104, cast(sum(profit#31)#102 as decimal(38,2)) AS profit#105] - -(86) ReusedExchange [Reuses operator id: 83] -Output [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] - -(87) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] - -(88) HashAggregate [codegen id : 16] -Input [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Keys [2]: [channel#106, id#107] -Functions [3]: [sum(sales#114), sum(returns#115), sum(profit#116)] -Aggregate Attributes [3]: [sum(sales#114)#100, sum(returns#115)#101, sum(profit#116)#102] -Results [4]: [channel#106, sum(sales#114)#100 AS sales#117, sum(returns#115)#101 AS returns#118, sum(profit#116)#102 AS profit#119] - -(89) HashAggregate [codegen id : 16] -Input [4]: [channel#106, sales#117, returns#118, profit#119] -Keys [1]: [channel#106] -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#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(90) CometColumnarExchange -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(91) CometColumnarToRow [codegen id : 17] -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(92) HashAggregate [codegen id : 17] -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#106] -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#106, 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] - -(93) ReusedExchange [Reuses operator id: 83] -Output [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(94) CometColumnarToRow [codegen id : 25] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(95) HashAggregate [codegen id : 25] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [2]: [channel#139, id#140] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#100, sum(returns#148)#101, sum(profit#149)#102] -Results [3]: [sum(sales#147)#100 AS sales#150, sum(returns#148)#101 AS returns#151, sum(profit#149)#102 AS profit#152] - -(96) HashAggregate [codegen id : 25] -Input [3]: [sales#150, returns#151, profit#152] +(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#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] + +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] + +(86) HashAggregate [codegen id : 12] +Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#102), sum(returns#103), sum(profit#104)] +Aggregate Attributes [3]: [sum(sales#102)#88, sum(returns#103)#89, sum(profit#104)#90] +Results [4]: [channel#94, sum(sales#102)#88 AS sales#105, sum(returns#103)#89 AS returns#106, sum(profit#104)#90 AS profit#107] + +(87) HashAggregate [codegen id : 12] +Input [4]: [channel#94, sales#105, returns#106, profit#107] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#105), partial_sum(returns#106), partial_sum(profit#107)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] + +(88) CometColumnarExchange +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] + +(90) HashAggregate [codegen id : 13] +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] +Aggregate Attributes [3]: [sum(sales#105)#120, sum(returns#106)#121, sum(profit#107)#122] +Results [5]: [channel#94, null AS id#123, sum(sales#105)#120 AS sales#124, sum(returns#106)#121 AS returns#125, sum(profit#107)#122 AS profit#126] + +(91) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] + +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] + +(93) HashAggregate [codegen id : 19] +Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [2]: [channel#127, id#128] +Functions [3]: [sum(sales#135), sum(returns#136), sum(profit#137)] +Aggregate Attributes [3]: [sum(sales#135)#88, sum(returns#136)#89, sum(profit#137)#90] +Results [3]: [sum(sales#135)#88 AS sales#138, sum(returns#136)#89 AS returns#139, sum(profit#137)#90 AS profit#140] + +(94) HashAggregate [codegen id : 19] +Input [3]: [sales#138, returns#139, profit#140] Keys: [] -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 [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Functions [3]: [partial_sum(sales#138), partial_sum(returns#139), partial_sum(profit#140)] +Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Results [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(97) CometColumnarExchange -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(95) CometColumnarExchange +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(98) CometColumnarToRow [codegen id : 26] -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +(96) CometColumnarToRow [codegen id : 20] +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(99) HashAggregate [codegen id : 26] -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +(97) HashAggregate [codegen id : 20] +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] Keys: [] -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]: [null AS channel#168, null AS id#169, sum(sales#150)#165 AS sales#170, sum(returns#151)#166 AS returns#171, sum(profit#152)#167 AS profit#172] +Functions [3]: [sum(sales#138), sum(returns#139), sum(profit#140)] +Aggregate Attributes [3]: [sum(sales#138)#153, sum(returns#139)#154, sum(profit#140)#155] +Results [5]: [null AS channel#156, null AS id#157, sum(sales#138)#153 AS sales#158, sum(returns#139)#154 AS returns#159, sum(profit#140)#155 AS profit#160] -(100) Union +(98) Union -(101) HashAggregate [codegen id : 27] -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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#28, id#29, sales#103, returns#104, profit#105] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(102) CometColumnarExchange -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Arguments: hashpartitioning(channel#28, id#29, sales#103, returns#104, profit#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(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] -(103) CometHashAggregate -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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: [] -(104) CometTakeOrderedAndProject -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#28 ASC NULLS FIRST,id#29 ASC NULLS FIRST], output=[channel#28,id#29,sales#103,returns#104,profit#105]), [channel#28, id#29, sales#103, returns#104, profit#105], 100, 0, [channel#28 ASC NULLS FIRST, id#29 ASC NULLS FIRST], [channel#28, id#29, sales#103, returns#104, profit#105] +(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] -(105) CometColumnarToRow [codegen id : 28] -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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 (110) -+- * CometColumnarToRow (109) - +- CometProject (108) - +- CometFilter (107) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (106) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(106) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(107) CometFilter +(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)) -(108) CometProject +(106) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(109) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] -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=12] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 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#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 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-spark4_0/q77a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt index f58252a7eb..3b44c6c27c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/extended.txt @@ -9,34 +9,33 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : : +- CometColumnarToRow + : :- 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 @@ -56,8 +55,8 @@ CometColumnarToRow : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject @@ -68,8 +67,8 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -80,29 +79,28 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - : +- CometColumnarToRow + : +- 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 @@ -128,34 +126,33 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : : +- CometColumnarToRow + : :- 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 @@ -175,8 +172,8 @@ CometColumnarToRow : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject @@ -187,8 +184,8 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -199,29 +196,28 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - : +- CometColumnarToRow + : +- 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 @@ -247,34 +243,33 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : +- CometColumnarToRow + :- 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 @@ -294,8 +289,8 @@ CometColumnarToRow :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -306,8 +301,8 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -318,29 +313,28 @@ CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - +- CometColumnarToRow + +- 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 @@ -358,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 251 out of 332 eligible operators (75%). Final plan contains 27 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-spark4_0/q77a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt index fba0dcde79..ac3d312ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a.native_iceberg_compat/simplified.txt @@ -1,28 +1,28 @@ -WholeStageCodegen (28) +WholeStageCodegen (22) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (27) + WholeStageCodegen (21) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (8) + 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 (7) + WholeStageCodegen (5) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (2) - 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 + 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] @@ -46,32 +46,28 @@ WholeStageCodegen (28) CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - 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 (4) + 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 #9 - WholeStageCodegen (3) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + 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] @@ -79,9 +75,9 @@ WholeStageCodegen (28) 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 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + 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] @@ -89,12 +85,12 @@ WholeStageCodegen (28) 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 (6) - 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 + 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] @@ -108,40 +104,36 @@ WholeStageCodegen (28) 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] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (5) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - 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 (17) + 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] #16 - WholeStageCodegen (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 (26) + 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 #17 - WholeStageCodegen (25) + 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-spark4_0/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt index e1a205bd43..9554c6732e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt @@ -1,109 +1,107 @@ == Physical Plan == -* CometColumnarToRow (105) -+- CometTakeOrderedAndProject (104) - +- CometHashAggregate (103) - +- CometColumnarExchange (102) - +- * HashAggregate (101) - +- Union (100) - :- * HashAggregate (85) - : +- * CometColumnarToRow (84) - : +- CometColumnarExchange (83) - : +- * HashAggregate (82) - : +- Union (81) - : :- * Project (32) - : : +- * BroadcastHashJoin LeftOuter BuildRight (31) - : : :- * HashAggregate (17) - : : : +- * CometColumnarToRow (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) - : : +- BroadcastExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometProject (25) - : : +- CometBroadcastHashJoin (24) - : : :- CometProject (22) - : : : +- CometBroadcastHashJoin (21) - : : : :- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (18) - : : : +- ReusedExchange (20) - : : +- ReusedExchange (23) - : :- * Project (51) - : : +- * BroadcastNestedLoopJoin Inner BuildLeft (50) - : : :- BroadcastExchange (41) - : : : +- * HashAggregate (40) - : : : +- * CometColumnarToRow (39) - : : : +- CometExchange (38) - : : : +- CometHashAggregate (37) - : : : +- CometProject (36) - : : : +- CometBroadcastHashJoin (35) - : : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (33) - : : : +- ReusedExchange (34) - : : +- * HashAggregate (49) - : : +- * CometColumnarToRow (48) - : : +- CometExchange (47) - : : +- CometHashAggregate (46) - : : +- CometProject (45) - : : +- CometBroadcastHashJoin (44) - : : :- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (42) - : : +- ReusedExchange (43) - : +- * Project (80) - : +- * BroadcastHashJoin LeftOuter BuildRight (79) - : :- * HashAggregate (65) - : : +- * CometColumnarToRow (64) - : : +- CometExchange (63) - : : +- CometHashAggregate (62) - : : +- CometProject (61) - : : +- CometBroadcastHashJoin (60) - : : :- CometProject (56) - : : : +- CometBroadcastHashJoin (55) - : : : :- CometFilter (53) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (52) - : : : +- ReusedExchange (54) - : : +- CometBroadcastExchange (59) - : : +- CometFilter (58) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page (57) - : +- BroadcastExchange (78) - : +- * HashAggregate (77) - : +- * CometColumnarToRow (76) - : +- CometExchange (75) - : +- CometHashAggregate (74) - : +- CometProject (73) - : +- CometBroadcastHashJoin (72) - : :- CometProject (70) - : : +- CometBroadcastHashJoin (69) - : : :- CometFilter (67) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (66) - : : +- ReusedExchange (68) - : +- ReusedExchange (71) - :- * HashAggregate (92) - : +- * CometColumnarToRow (91) - : +- CometColumnarExchange (90) - : +- * HashAggregate (89) - : +- * HashAggregate (88) - : +- * CometColumnarToRow (87) - : +- ReusedExchange (86) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometColumnarExchange (97) - +- * HashAggregate (96) - +- * HashAggregate (95) - +- * CometColumnarToRow (94) - +- ReusedExchange (93) +* 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) + : +- * 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 @@ -179,465 +177,445 @@ Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(Un Input [3]: [s_store_sk#8, sum#9, sum#10] Arguments: hashpartitioning(s_store_sk#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(16) CometColumnarToRow [codegen id : 2] -Input [3]: [s_store_sk#8, sum#9, sum#10] - -(17) HashAggregate [codegen id : 2] +(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))#11, sum(UnscaledValue(ss_net_profit#3))#12] -Results [3]: [s_store_sk#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#11,17,2) AS sales#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#12,17,2) AS profit#14] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns -Output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(sr_returned_date_sk#18), dynamicpruningexpression(sr_returned_date_sk#18 IN dynamicpruning#19)] +PartitionFilters: [isnotnull(sr_returned_date_sk#14), dynamicpruningexpression(sr_returned_date_sk#14 IN dynamicpruning#15)] PushedFilters: [IsNotNull(sr_store_sk)] ReadSchema: struct -(19) CometFilter -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Condition : isnotnull(sr_store_sk#15) - -(20) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#20] - -(21) CometBroadcastHashJoin -Left output [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18] -Right output [1]: [d_date_sk#20] -Arguments: [sr_returned_date_sk#18], [d_date_sk#20], Inner, BuildRight - -(22) CometProject -Input [5]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, sr_returned_date_sk#18, d_date_sk#20] -Arguments: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17], [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] - -(23) ReusedExchange [Reuses operator id: 11] -Output [1]: [s_store_sk#21] - -(24) CometBroadcastHashJoin -Left output [3]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17] -Right output [1]: [s_store_sk#21] -Arguments: [sr_store_sk#15], [s_store_sk#21], Inner, BuildRight - -(25) CometProject -Input [4]: [sr_store_sk#15, sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Arguments: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21], [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] - -(26) CometHashAggregate -Input [3]: [sr_return_amt#16, sr_net_loss#17, s_store_sk#21] -Keys [1]: [s_store_sk#21] -Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#16)), partial_sum(UnscaledValue(sr_net_loss#17))] - -(27) CometExchange -Input [3]: [s_store_sk#21, sum#22, sum#23] -Arguments: hashpartitioning(s_store_sk#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] - -(28) CometColumnarToRow [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] - -(29) HashAggregate [codegen id : 1] -Input [3]: [s_store_sk#21, sum#22, sum#23] -Keys [1]: [s_store_sk#21] -Functions [2]: [sum(UnscaledValue(sr_return_amt#16)), sum(UnscaledValue(sr_net_loss#17))] -Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#16))#24, sum(UnscaledValue(sr_net_loss#17))#25] -Results [3]: [s_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#16))#24,17,2) AS returns#26, MakeDecimal(sum(UnscaledValue(sr_net_loss#17))#25,17,2) AS profit_loss#27] - -(30) BroadcastExchange -Input [3]: [s_store_sk#21, returns#26, profit_loss#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] - -(31) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [s_store_sk#8] -Right keys [1]: [s_store_sk#21] -Join type: LeftOuter -Join condition: None +(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] + +(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) 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) 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] + +(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))] -(32) Project [codegen id : 2] -Output [5]: [store channel AS channel#28, s_store_sk#8 AS id#29, sales#13, coalesce(returns#26, 0.00) AS returns#30, (profit#14 - coalesce(profit_loss#27, 0.00)) AS profit#31] -Input [6]: [s_store_sk#8, sales#13, profit#14, s_store_sk#21, returns#26, profit_loss#27] +(28) CometBroadcastExchange +Input [3]: [s_store_sk#17, returns#20, profit_loss#21] +Arguments: [s_store_sk#17, returns#20, profit_loss#21] -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] +(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 + +(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] + +(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#35), dynamicpruningexpression(cs_sold_date_sk#35 IN dynamicpruning#36)] +PartitionFilters: [isnotnull(cs_sold_date_sk#31), dynamicpruningexpression(cs_sold_date_sk#31 IN dynamicpruning#32)] ReadSchema: struct -(34) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#37] +(33) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#33] -(35) CometBroadcastHashJoin -Left output [4]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35] -Right output [1]: [d_date_sk#37] -Arguments: [cs_sold_date_sk#35], [d_date_sk#37], Inner, BuildRight +(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) CometProject -Input [5]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34, cs_sold_date_sk#35, d_date_sk#37] -Arguments: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34], [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] +(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] -(37) CometHashAggregate -Input [3]: [cs_call_center_sk#32, cs_ext_sales_price#33, cs_net_profit#34] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#33)), partial_sum(UnscaledValue(cs_net_profit#34))] +(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))] -(38) CometExchange -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Arguments: hashpartitioning(cs_call_center_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(39) CometColumnarToRow [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] +(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))] -(40) HashAggregate [codegen id : 3] -Input [3]: [cs_call_center_sk#32, sum#38, sum#39] -Keys [1]: [cs_call_center_sk#32] -Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#33)), sum(UnscaledValue(cs_net_profit#34))] -Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#33))#40, sum(UnscaledValue(cs_net_profit#34))#41] -Results [3]: [cs_call_center_sk#32, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#33))#40,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#41,17,2) AS profit#43] +(39) CometColumnarToRow [codegen id : 2] +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] -(41) BroadcastExchange -Input [3]: [cs_call_center_sk#32, sales#42, profit#43] -Arguments: IdentityBroadcastMode, [plan_id=5] +(40) BroadcastExchange +Input [3]: [cs_call_center_sk#28, sales#36, profit#37] +Arguments: IdentityBroadcastMode, [plan_id=4] -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] +(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#46), dynamicpruningexpression(cr_returned_date_sk#46 IN dynamicpruning#47)] +PartitionFilters: [isnotnull(cr_returned_date_sk#40), dynamicpruningexpression(cr_returned_date_sk#40 IN dynamicpruning#41)] ReadSchema: struct -(43) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#48] +(42) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#42] -(44) CometBroadcastHashJoin -Left output [3]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46] -Right output [1]: [d_date_sk#48] -Arguments: [cr_returned_date_sk#46], [d_date_sk#48], Inner, BuildRight +(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) CometProject -Input [4]: [cr_return_amount#44, cr_net_loss#45, cr_returned_date_sk#46, d_date_sk#48] -Arguments: [cr_return_amount#44, cr_net_loss#45], [cr_return_amount#44, cr_net_loss#45] +(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) CometHashAggregate -Input [2]: [cr_return_amount#44, cr_net_loss#45] +(45) CometHashAggregate +Input [2]: [cr_return_amount#38, cr_net_loss#39] Keys: [] -Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#44)), partial_sum(UnscaledValue(cr_net_loss#45))] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#38)), partial_sum(UnscaledValue(cr_net_loss#39))] -(47) CometExchange -Input [2]: [sum#49, sum#50] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(46) CometExchange +Input [2]: [sum#43, sum#44] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometColumnarToRow -Input [2]: [sum#49, sum#50] - -(49) HashAggregate -Input [2]: [sum#49, sum#50] +(47) CometHashAggregate +Input [2]: [sum#43, sum#44] Keys: [] -Functions [2]: [sum(UnscaledValue(cr_return_amount#44)), sum(UnscaledValue(cr_net_loss#45))] -Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#44))#51, sum(UnscaledValue(cr_net_loss#45))#52] -Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#44))#51,17,2) AS returns#53, MakeDecimal(sum(UnscaledValue(cr_net_loss#45))#52,17,2) AS profit_loss#54] +Functions [2]: [sum(UnscaledValue(cr_return_amount#38)), sum(UnscaledValue(cr_net_loss#39))] + +(48) CometColumnarToRow +Input [2]: [returns#45, profit_loss#46] -(50) BroadcastNestedLoopJoin [codegen id : 4] +(49) BroadcastNestedLoopJoin [codegen id : 3] Join type: Inner Join condition: None -(51) Project [codegen id : 4] -Output [5]: [catalog channel AS channel#55, cs_call_center_sk#32 AS id#56, sales#42, returns#53, (profit#43 - profit_loss#54) AS profit#57] -Input [5]: [cs_call_center_sk#32, sales#42, profit#43, returns#53, profit_loss#54] +(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] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#61), dynamicpruningexpression(ws_sold_date_sk#61 IN dynamicpruning#62)] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] PushedFilters: [IsNotNull(ws_web_page_sk)] ReadSchema: struct -(53) CometFilter -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Condition : isnotnull(ws_web_page_sk#58) +(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) -(54) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#63] +(53) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#55] -(55) CometBroadcastHashJoin -Left output [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61] -Right output [1]: [d_date_sk#63] -Arguments: [ws_sold_date_sk#61], [d_date_sk#63], Inner, BuildRight +(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 -(56) CometProject -Input [5]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, ws_sold_date_sk#61, d_date_sk#63] -Arguments: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60], [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] +(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] -(57) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Output [1]: [wp_web_page_sk#64] +(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 -(58) CometFilter -Input [1]: [wp_web_page_sk#64] -Condition : isnotnull(wp_web_page_sk#64) - -(59) CometBroadcastExchange -Input [1]: [wp_web_page_sk#64] -Arguments: [wp_web_page_sk#64] +(57) CometFilter +Input [1]: [wp_web_page_sk#56] +Condition : isnotnull(wp_web_page_sk#56) -(60) CometBroadcastHashJoin -Left output [3]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60] -Right output [1]: [wp_web_page_sk#64] -Arguments: [ws_web_page_sk#58], [wp_web_page_sk#64], Inner, BuildRight +(58) CometBroadcastExchange +Input [1]: [wp_web_page_sk#56] +Arguments: [wp_web_page_sk#56] -(61) CometProject -Input [4]: [ws_web_page_sk#58, ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Arguments: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64], [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] +(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 -(62) CometHashAggregate -Input [3]: [ws_ext_sales_price#59, ws_net_profit#60, wp_web_page_sk#64] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#59)), partial_sum(UnscaledValue(ws_net_profit#60))] +(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] -(63) CometExchange -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Arguments: hashpartitioning(wp_web_page_sk#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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))] -(64) CometColumnarToRow [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] +(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) HashAggregate [codegen id : 6] -Input [3]: [wp_web_page_sk#64, sum#65, sum#66] -Keys [1]: [wp_web_page_sk#64] -Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#59)), sum(UnscaledValue(ws_net_profit#60))] -Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#59))#67, sum(UnscaledValue(ws_net_profit#60))#68] -Results [3]: [wp_web_page_sk#64, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#59))#67,17,2) AS sales#69, MakeDecimal(sum(UnscaledValue(ws_net_profit#60))#68,17,2) AS profit#70] +(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))] -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] +(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#74), dynamicpruningexpression(wr_returned_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(wr_returned_date_sk#62), dynamicpruningexpression(wr_returned_date_sk#62 IN dynamicpruning#63)] PushedFilters: [IsNotNull(wr_web_page_sk)] ReadSchema: struct -(67) CometFilter -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Condition : isnotnull(wr_web_page_sk#71) +(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) -(68) ReusedExchange [Reuses operator id: 6] -Output [1]: [d_date_sk#76] +(66) ReusedExchange [Reuses operator id: 6] +Output [1]: [d_date_sk#64] -(69) CometBroadcastHashJoin -Left output [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74] -Right output [1]: [d_date_sk#76] -Arguments: [wr_returned_date_sk#74], [d_date_sk#76], Inner, BuildRight +(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 -(70) CometProject -Input [5]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wr_returned_date_sk#74, d_date_sk#76] -Arguments: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73], [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] +(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] -(71) ReusedExchange [Reuses operator id: 59] -Output [1]: [wp_web_page_sk#77] +(69) ReusedExchange [Reuses operator id: 58] +Output [1]: [wp_web_page_sk#65] -(72) CometBroadcastHashJoin -Left output [3]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73] -Right output [1]: [wp_web_page_sk#77] -Arguments: [wr_web_page_sk#71], [wp_web_page_sk#77], Inner, BuildRight +(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 -(73) CometProject -Input [4]: [wr_web_page_sk#71, wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Arguments: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77], [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] +(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] -(74) CometHashAggregate -Input [3]: [wr_return_amt#72, wr_net_loss#73, wp_web_page_sk#77] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#72)), partial_sum(UnscaledValue(wr_net_loss#73))] - -(75) CometExchange -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Arguments: hashpartitioning(wp_web_page_sk#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(76) CometColumnarToRow [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] - -(77) HashAggregate [codegen id : 5] -Input [3]: [wp_web_page_sk#77, sum#78, sum#79] -Keys [1]: [wp_web_page_sk#77] -Functions [2]: [sum(UnscaledValue(wr_return_amt#72)), sum(UnscaledValue(wr_net_loss#73))] -Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#72))#80, sum(UnscaledValue(wr_net_loss#73))#81] -Results [3]: [wp_web_page_sk#77, MakeDecimal(sum(UnscaledValue(wr_return_amt#72))#80,17,2) AS returns#82, MakeDecimal(sum(UnscaledValue(wr_net_loss#73))#81,17,2) AS profit_loss#83] - -(78) BroadcastExchange -Input [3]: [wp_web_page_sk#77, returns#82, profit_loss#83] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] - -(79) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [wp_web_page_sk#64] -Right keys [1]: [wp_web_page_sk#77] -Join type: LeftOuter -Join condition: None +(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] -(80) Project [codegen id : 6] -Output [5]: [web channel AS channel#84, wp_web_page_sk#64 AS id#85, sales#69, coalesce(returns#82, 0.00) AS returns#86, (profit#70 - coalesce(profit_loss#83, 0.00)) AS profit#87] -Input [6]: [wp_web_page_sk#64, sales#69, profit#70, wp_web_page_sk#77, returns#82, profit_loss#83] - -(81) Union - -(82) HashAggregate [codegen id : 7] -Input [5]: [channel#28, id#29, sales#13, returns#30, profit#31] -Keys [2]: [channel#28, id#29] -Functions [3]: [partial_sum(sales#13), partial_sum(returns#30), partial_sum(profit#31)] -Aggregate Attributes [6]: [sum#88, isEmpty#89, sum#90, isEmpty#91, sum#92, isEmpty#93] -Results [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] - -(83) CometColumnarExchange -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Arguments: hashpartitioning(channel#28, id#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(84) CometColumnarToRow [codegen id : 8] -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] - -(85) HashAggregate [codegen id : 8] -Input [8]: [channel#28, id#29, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] -Keys [2]: [channel#28, id#29] -Functions [3]: [sum(sales#13), sum(returns#30), sum(profit#31)] -Aggregate Attributes [3]: [sum(sales#13)#100, sum(returns#30)#101, sum(profit#31)#102] -Results [5]: [channel#28, id#29, cast(sum(sales#13)#100 as decimal(37,2)) AS sales#103, cast(sum(returns#30)#101 as decimal(37,2)) AS returns#104, cast(sum(profit#31)#102 as decimal(38,2)) AS profit#105] - -(86) ReusedExchange [Reuses operator id: 83] -Output [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] - -(87) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] - -(88) HashAggregate [codegen id : 16] -Input [8]: [channel#106, id#107, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] -Keys [2]: [channel#106, id#107] -Functions [3]: [sum(sales#114), sum(returns#115), sum(profit#116)] -Aggregate Attributes [3]: [sum(sales#114)#100, sum(returns#115)#101, sum(profit#116)#102] -Results [4]: [channel#106, sum(sales#114)#100 AS sales#117, sum(returns#115)#101 AS returns#118, sum(profit#116)#102 AS profit#119] - -(89) HashAggregate [codegen id : 16] -Input [4]: [channel#106, sales#117, returns#118, profit#119] -Keys [1]: [channel#106] -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#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(90) CometColumnarExchange -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Arguments: hashpartitioning(channel#106, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(91) CometColumnarToRow [codegen id : 17] -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] - -(92) HashAggregate [codegen id : 17] -Input [7]: [channel#106, sum#126, isEmpty#127, sum#128, isEmpty#129, sum#130, isEmpty#131] -Keys [1]: [channel#106] -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#106, 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] - -(93) ReusedExchange [Reuses operator id: 83] -Output [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(94) CometColumnarToRow [codegen id : 25] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] - -(95) HashAggregate [codegen id : 25] -Input [8]: [channel#139, id#140, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Keys [2]: [channel#139, id#140] -Functions [3]: [sum(sales#147), sum(returns#148), sum(profit#149)] -Aggregate Attributes [3]: [sum(sales#147)#100, sum(returns#148)#101, sum(profit#149)#102] -Results [3]: [sum(sales#147)#100 AS sales#150, sum(returns#148)#101 AS returns#151, sum(profit#149)#102 AS profit#152] - -(96) HashAggregate [codegen id : 25] -Input [3]: [sales#150, returns#151, profit#152] +(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#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] + +(85) CometColumnarToRow [codegen id : 12] +Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] + +(86) HashAggregate [codegen id : 12] +Input [8]: [channel#94, id#95, sum#96, isEmpty#97, sum#98, isEmpty#99, sum#100, isEmpty#101] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#102), sum(returns#103), sum(profit#104)] +Aggregate Attributes [3]: [sum(sales#102)#88, sum(returns#103)#89, sum(profit#104)#90] +Results [4]: [channel#94, sum(sales#102)#88 AS sales#105, sum(returns#103)#89 AS returns#106, sum(profit#104)#90 AS profit#107] + +(87) HashAggregate [codegen id : 12] +Input [4]: [channel#94, sales#105, returns#106, profit#107] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#105), partial_sum(returns#106), partial_sum(profit#107)] +Aggregate Attributes [6]: [sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Results [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] + +(88) CometColumnarExchange +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(89) CometColumnarToRow [codegen id : 13] +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] + +(90) HashAggregate [codegen id : 13] +Input [7]: [channel#94, sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] +Aggregate Attributes [3]: [sum(sales#105)#120, sum(returns#106)#121, sum(profit#107)#122] +Results [5]: [channel#94, null AS id#123, sum(sales#105)#120 AS sales#124, sum(returns#106)#121 AS returns#125, sum(profit#107)#122 AS profit#126] + +(91) ReusedExchange [Reuses operator id: 81] +Output [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] + +(92) CometColumnarToRow [codegen id : 19] +Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] + +(93) HashAggregate [codegen id : 19] +Input [8]: [channel#127, id#128, sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Keys [2]: [channel#127, id#128] +Functions [3]: [sum(sales#135), sum(returns#136), sum(profit#137)] +Aggregate Attributes [3]: [sum(sales#135)#88, sum(returns#136)#89, sum(profit#137)#90] +Results [3]: [sum(sales#135)#88 AS sales#138, sum(returns#136)#89 AS returns#139, sum(profit#137)#90 AS profit#140] + +(94) HashAggregate [codegen id : 19] +Input [3]: [sales#138, returns#139, profit#140] Keys: [] -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 [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +Functions [3]: [partial_sum(sales#138), partial_sum(returns#139), partial_sum(profit#140)] +Aggregate Attributes [6]: [sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] +Results [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(97) CometColumnarExchange -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +(95) CometColumnarExchange +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] -(98) CometColumnarToRow [codegen id : 26] -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +(96) CometColumnarToRow [codegen id : 20] +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] -(99) HashAggregate [codegen id : 26] -Input [6]: [sum#159, isEmpty#160, sum#161, isEmpty#162, sum#163, isEmpty#164] +(97) HashAggregate [codegen id : 20] +Input [6]: [sum#147, isEmpty#148, sum#149, isEmpty#150, sum#151, isEmpty#152] Keys: [] -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]: [null AS channel#168, null AS id#169, sum(sales#150)#165 AS sales#170, sum(returns#151)#166 AS returns#171, sum(profit#152)#167 AS profit#172] +Functions [3]: [sum(sales#138), sum(returns#139), sum(profit#140)] +Aggregate Attributes [3]: [sum(sales#138)#153, sum(returns#139)#154, sum(profit#140)#155] +Results [5]: [null AS channel#156, null AS id#157, sum(sales#138)#153 AS sales#158, sum(returns#139)#154 AS returns#159, sum(profit#140)#155 AS profit#160] -(100) Union +(98) Union -(101) HashAggregate [codegen id : 27] -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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#28, id#29, sales#103, returns#104, profit#105] +Results [5]: [channel#24, id#25, sales#91, returns#92, profit#93] -(102) CometColumnarExchange -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Arguments: hashpartitioning(channel#28, id#29, sales#103, returns#104, profit#105, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(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] -(103) CometHashAggregate -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Keys [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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: [] -(104) CometTakeOrderedAndProject -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#28 ASC NULLS FIRST,id#29 ASC NULLS FIRST], output=[channel#28,id#29,sales#103,returns#104,profit#105]), [channel#28, id#29, sales#103, returns#104, profit#105], 100, 0, [channel#28 ASC NULLS FIRST, id#29 ASC NULLS FIRST], [channel#28, id#29, sales#103, returns#104, profit#105] +(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] -(105) CometColumnarToRow [codegen id : 28] -Input [5]: [channel#28, id#29, sales#103, returns#104, profit#105] +(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 (110) -+- * CometColumnarToRow (109) - +- CometProject (108) - +- CometFilter (107) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (106) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(106) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(107) CometFilter +(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)) -(108) CometProject +(106) CometProject Input [2]: [d_date_sk#6, d_date#7] Arguments: [d_date_sk#6], [d_date_sk#6] -(109) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#6] -(110) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#6] -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=12] -Subquery:2 Hosting operator id = 18 Hosting Expression = sr_returned_date_sk#18 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#35 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 32 Hosting Expression = cs_sold_date_sk#31 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 42 Hosting Expression = cr_returned_date_sk#46 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 41 Hosting Expression = cr_returned_date_sk#40 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#61 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 51 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 66 Hosting Expression = wr_returned_date_sk#74 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-spark4_0/q77a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt index f58252a7eb..3b44c6c27c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/extended.txt @@ -9,34 +9,33 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : : +- CometColumnarToRow + : :- 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 @@ -56,8 +55,8 @@ CometColumnarToRow : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject @@ -68,8 +67,8 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -80,29 +79,28 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - : +- CometColumnarToRow + : +- 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 @@ -128,34 +126,33 @@ CometColumnarToRow : +- CometColumnarExchange : +- HashAggregate : +- Union - : :- Project - : : +- BroadcastHashJoin - : : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : : +- CometColumnarToRow - : : : +- 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 - : : +- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : : +- CometColumnarToRow + : :- 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 @@ -175,8 +172,8 @@ CometColumnarToRow : :- Project : : +- BroadcastNestedLoopJoin : : :- BroadcastExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : : +- CometColumnarToRow + : : : +- CometColumnarToRow + : : : +- CometHashAggregate : : : +- CometExchange : : : +- CometHashAggregate : : : +- CometProject @@ -187,8 +184,8 @@ CometColumnarToRow : : : +- CometProject : : : +- CometFilter : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -199,29 +196,28 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - : +- CometColumnarToRow + : +- 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 @@ -247,34 +243,33 @@ CometColumnarToRow +- CometColumnarExchange +- HashAggregate +- Union - :- Project - : +- BroadcastHashJoin - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#1, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#3,17,2) AS sales#4, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#6,17,2) AS profit#7)] - : : +- CometColumnarToRow - : : +- 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 - : +- BroadcastExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(s_store_sk#8, MakeDecimal(sum(UnscaledValue(sr_return_amt#9))#10,17,2) AS returns#11, MakeDecimal(sum(UnscaledValue(sr_net_loss#12))#13,17,2) AS profit_loss#14)] - : +- CometColumnarToRow + :- 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 @@ -294,8 +289,8 @@ CometColumnarToRow :- Project : +- BroadcastNestedLoopJoin : :- BroadcastExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cs_call_center_sk#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#16))#17,17,2) AS sales#18, MakeDecimal(sum(UnscaledValue(cs_net_profit#19))#20,17,2) AS profit#21)] - : : +- CometColumnarToRow + : : +- CometColumnarToRow + : : +- CometHashAggregate : : +- CometExchange : : +- CometHashAggregate : : +- CometProject @@ -306,8 +301,8 @@ CometColumnarToRow : : +- CometProject : : +- CometFilter : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(cr_return_amount#22))#23,17,2) AS returns#24, MakeDecimal(sum(UnscaledValue(cr_net_loss#25))#26,17,2) AS profit_loss#27)] - : +- CometColumnarToRow + : +- CometColumnarToRow + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate : +- CometProject @@ -318,29 +313,28 @@ CometColumnarToRow : +- CometProject : +- CometFilter : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- BroadcastHashJoin - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#28, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#29))#30,17,2) AS sales#31, MakeDecimal(sum(UnscaledValue(ws_net_profit#32))#33,17,2) AS profit#34)] - : +- CometColumnarToRow - : +- 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 - +- BroadcastExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(wp_web_page_sk#35, MakeDecimal(sum(UnscaledValue(wr_return_amt#36))#37,17,2) AS returns#38, MakeDecimal(sum(UnscaledValue(wr_net_loss#39))#40,17,2) AS profit_loss#41)] - +- CometColumnarToRow + +- 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 @@ -358,4 +352,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_page -Comet accelerated 251 out of 332 eligible operators (75%). Final plan contains 27 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-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt index fba0dcde79..ac3d312ee8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -1,28 +1,28 @@ -WholeStageCodegen (28) +WholeStageCodegen (22) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (27) + WholeStageCodegen (21) HashAggregate [channel,id,sales,returns,profit] InputAdapter Union - WholeStageCodegen (8) + 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 (7) + WholeStageCodegen (5) HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter Union - WholeStageCodegen (2) - 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 + 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] @@ -46,32 +46,28 @@ WholeStageCodegen (28) CometBroadcastExchange [s_store_sk] #6 CometFilter [s_store_sk] CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (1) - HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - 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 (4) + 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 #9 - WholeStageCodegen (3) - HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] - CometColumnarToRow - InputAdapter + 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] @@ -79,9 +75,9 @@ WholeStageCodegen (28) 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 - HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter + 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] @@ -89,12 +85,12 @@ WholeStageCodegen (28) 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 (6) - 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 + 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] @@ -108,40 +104,36 @@ WholeStageCodegen (28) 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] - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (5) - HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] - CometColumnarToRow - InputAdapter - 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 (17) + 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] #16 - WholeStageCodegen (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 (26) + 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 #17 - WholeStageCodegen (25) + 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-spark4_0/q78.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt index 8395b09b2f..82a5357f89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/explain.txt @@ -1,73 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (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) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- 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 (58) +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 @@ -159,263 +157,249 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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) CometColumnarToRow [codegen id : 1] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] - -(21) HashAggregate [codegen id : 1] +(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)#17, sum(UnscaledValue(ss_wholesale_cost#5))#18, sum(UnscaledValue(ss_sales_price#6))#19] -Results [6]: [d_year#13 AS ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#17 AS ss_qty#21, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#18,17,2) AS ss_wc#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#19,17,2) AS ss_sp#23] -(22) Sort [codegen id : 1] -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20 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] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(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] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(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)) -(25) CometExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(26) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) - -(29) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] - -(30) CometExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(31) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] - -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter - -(33) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) - -(34) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] - -(35) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#35, d_year#36] - -(36) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [d_date_sk#35, d_year#36] -Arguments: [ws_sold_date_sk#30], [d_date_sk#35], Inner, BuildRight - -(37) CometProject -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] - -(38) CometHashAggregate -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] - -(39) CometExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] - -(41) HashAggregate [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum(ws_quantity#27)#40, sum(UnscaledValue(ws_wholesale_cost#28))#41, sum(UnscaledValue(ws_sales_price#29))#42] -Results [6]: [d_year#36 AS ws_sold_year#43, ws_item_sk#24, ws_bill_customer_sk#25 AS ws_customer_sk#44, sum(ws_quantity#27)#40 AS ws_qty#45, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#28))#41,17,2) AS ws_wc#46, MakeDecimal(sum(UnscaledValue(ws_sales_price#29))#42,17,2) AS ws_sp#47] - -(42) Filter [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) Sort [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 3] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 3] -Output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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)) + +(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] + +(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] + +(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] + +(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 + +(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) + +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 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#56, cr_order_number#57, cr_returned_date_sk#58] +(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#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(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)) + +(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] + +(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] -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(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] -(53) CometExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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 -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(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) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(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] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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 -(58) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#59, d_year#60] +(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] -(59) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [d_date_sk#59, d_year#60] -Arguments: [cs_sold_date_sk#54], [d_date_sk#59], Inner, BuildRight +(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))] -(60) CometProject -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +(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] (61) CometHashAggregate -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] - -(62) CometExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometColumnarToRow [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] - -(64) HashAggregate [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum(cs_quantity#51)#64, sum(UnscaledValue(cs_wholesale_cost#52))#65, sum(UnscaledValue(cs_sales_price#53))#66] -Results [6]: [d_year#60 AS cs_sold_year#67, cs_item_sk#49, cs_bill_customer_sk#48 AS cs_customer_sk#68, sum(cs_quantity#51)#64 AS cs_qty#69, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#52))#65,17,2) AS cs_wc#70, MakeDecimal(sum(UnscaledValue(cs_sales_price#53))#66,17,2) AS cs_sp#71] - -(65) Filter [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(66) Sort [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST], false, 0 - -(67) SortMergeJoin [codegen id : 5] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 5] -Output [13]: [round((cast(ss_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)))), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] - -(69) TakeOrderedAndProject -Input [13]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: 100, [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, ratio#72 ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +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))] + +(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) + +(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] + +(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 + +(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] + +(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] + +(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 (73) -+- * CometColumnarToRow (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(70) 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 -(71) 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)) -(72) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) 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 = 23 Hosting Expression = ws_sold_date_sk#30 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#54 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-spark4_0/q78.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt index 6516f29483..9d5ba53be1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/extended.txt @@ -1,63 +1,61 @@ TakeOrderedAndProject -+- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1 AS ss_sold_year#2, ss_item_sk#3, ss_customer_sk#4, sum(ss_quantity#5)#6 AS ss_qty#7, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#8))#9,17,2) AS ss_wc#10, MakeDecimal(sum(UnscaledValue(ss_sales_price#11))#12,17,2) AS ss_sp#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- Sort - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#14 AS ws_sold_year#15, ws_item_sk#16, ws_bill_customer_sk#17 AS ws_customer_sk#18, sum(ws_quantity#19)#20 AS ws_qty#21, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#22))#23,17,2) AS ws_wc#24, MakeDecimal(sum(UnscaledValue(ws_sales_price#25))#26,17,2) AS ws_sp#27)] - : +- CometColumnarToRow - : +- 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 - +- Sort - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#28 AS cs_sold_year#29, cs_item_sk#30, cs_bill_customer_sk#31 AS cs_customer_sk#32, sum(cs_quantity#33)#34 AS cs_qty#35, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#36))#37,17,2) AS cs_wc#38, MakeDecimal(sum(UnscaledValue(cs_sales_price#39))#40,17,2) AS cs_sp#41)] - +- CometColumnarToRow ++- 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 @@ -79,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 59 out of 76 eligible operators (77%). Final plan contains 4 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-spark4_0/q78.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt index 850bab3c02..273db28e77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78.native_iceberg_compat/simplified.txt @@ -1,90 +1,78 @@ 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 (5) + 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] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometColumnarToRow InputAdapter - WholeStageCodegen (3) - 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 (1) - 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 - 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 - 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] - InputAdapter - WholeStageCodegen (2) - 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 - 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 - InputAdapter - WholeStageCodegen (4) - 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 - 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] + 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 + 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 [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] + 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-spark4_0/q78/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt index 8395b09b2f..82a5357f89 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt @@ -1,73 +1,71 @@ == Physical Plan == -TakeOrderedAndProject (69) -+- * Project (68) - +- * SortMergeJoin Inner (67) - :- * Project (45) - : +- * SortMergeJoin Inner (44) - : :- * Sort (22) - : : +- * HashAggregate (21) - : : +- * CometColumnarToRow (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) - : +- * Sort (43) - : +- * Filter (42) - : +- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometFilter (33) - : : +- CometSortMergeJoin (32) - : : :- CometSort (26) - : : : +- CometExchange (25) - : : : +- CometFilter (24) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (23) - : : +- CometSort (31) - : : +- CometExchange (30) - : : +- CometProject (29) - : : +- CometFilter (28) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (27) - : +- ReusedExchange (35) - +- * Sort (66) - +- * Filter (65) - +- * HashAggregate (64) - +- * CometColumnarToRow (63) - +- CometExchange (62) - +- CometHashAggregate (61) - +- CometProject (60) - +- CometBroadcastHashJoin (59) - :- 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 (58) +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 @@ -159,263 +157,249 @@ Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesa 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) CometColumnarToRow [codegen id : 1] -Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#14, sum#15, sum#16] - -(21) HashAggregate [codegen id : 1] +(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)#17, sum(UnscaledValue(ss_wholesale_cost#5))#18, sum(UnscaledValue(ss_sales_price#6))#19] -Results [6]: [d_year#13 AS ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#17 AS ss_qty#21, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#18,17,2) AS ss_wc#22, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#19,17,2) AS ss_sp#23] -(22) Sort [codegen id : 1] -Input [6]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: [ss_sold_year#20 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] -(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] +(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] 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#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(24) CometFilter -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Condition : (isnotnull(ws_item_sk#24) AND isnotnull(ws_bill_customer_sk#25)) +(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)) -(25) CometExchange -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: hashpartitioning(ws_order_number#26, ws_item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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] -(26) CometSort -Input [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_order_number#26 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST] +(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] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] +(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] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] ReadSchema: struct -(28) CometFilter -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Condition : (isnotnull(wr_order_number#33) AND isnotnull(wr_item_sk#32)) - -(29) CometProject -Input [3]: [wr_item_sk#32, wr_order_number#33, wr_returned_date_sk#34] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_item_sk#32, wr_order_number#33] - -(30) CometExchange -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: hashpartitioning(wr_order_number#33, wr_item_sk#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(31) CometSort -Input [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [wr_item_sk#32, wr_order_number#33], [wr_order_number#33 ASC NULLS FIRST, wr_item_sk#32 ASC NULLS FIRST] - -(32) CometSortMergeJoin -Left output [7]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [wr_item_sk#32, wr_order_number#33] -Arguments: [ws_order_number#26, ws_item_sk#24], [wr_order_number#33, wr_item_sk#32], LeftOuter - -(33) CometFilter -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Condition : isnull(wr_order_number#33) - -(34) CometProject -Input [9]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_order_number#26, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, wr_item_sk#32, wr_order_number#33] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] - -(35) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#35, d_year#36] - -(36) CometBroadcastHashJoin -Left output [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30] -Right output [2]: [d_date_sk#35, d_year#36] -Arguments: [ws_sold_date_sk#30], [d_date_sk#35], Inner, BuildRight - -(37) CometProject -Input [8]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, ws_sold_date_sk#30, d_date_sk#35, d_year#36] -Arguments: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36], [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] - -(38) CometHashAggregate -Input [6]: [ws_item_sk#24, ws_bill_customer_sk#25, ws_quantity#27, ws_wholesale_cost#28, ws_sales_price#29, d_year#36] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [partial_sum(ws_quantity#27), partial_sum(UnscaledValue(ws_wholesale_cost#28)), partial_sum(UnscaledValue(ws_sales_price#29))] - -(39) CometExchange -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Arguments: hashpartitioning(d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(40) CometColumnarToRow [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] - -(41) HashAggregate [codegen id : 2] -Input [6]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25, sum#37, sum#38, sum#39] -Keys [3]: [d_year#36, ws_item_sk#24, ws_bill_customer_sk#25] -Functions [3]: [sum(ws_quantity#27), sum(UnscaledValue(ws_wholesale_cost#28)), sum(UnscaledValue(ws_sales_price#29))] -Aggregate Attributes [3]: [sum(ws_quantity#27)#40, sum(UnscaledValue(ws_wholesale_cost#28))#41, sum(UnscaledValue(ws_sales_price#29))#42] -Results [6]: [d_year#36 AS ws_sold_year#43, ws_item_sk#24, ws_bill_customer_sk#25 AS ws_customer_sk#44, sum(ws_quantity#27)#40 AS ws_qty#45, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#28))#41,17,2) AS ws_wc#46, MakeDecimal(sum(UnscaledValue(ws_sales_price#29))#42,17,2) AS ws_sp#47] - -(42) Filter [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Condition : (coalesce(ws_qty#45, 0) > 0) - -(43) Sort [codegen id : 2] -Input [6]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] -Arguments: [ws_sold_year#43 ASC NULLS FIRST, ws_item_sk#24 ASC NULLS FIRST, ws_customer_sk#44 ASC NULLS FIRST], false, 0 - -(44) SortMergeJoin [codegen id : 3] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44] -Join type: Inner -Join condition: None - -(45) Project [codegen id : 3] -Output [9]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47] -Input [12]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_sold_year#43, ws_item_sk#24, ws_customer_sk#44, ws_qty#45, ws_wc#46, ws_sp#47] - -(46) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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)) + +(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] + +(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] + +(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] + +(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 + +(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) + +(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] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#54), dynamicpruningexpression(cs_sold_date_sk#54 IN dynamicpruning#55)] +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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Condition : (isnotnull(cs_item_sk#49) AND isnotnull(cs_bill_customer_sk#48)) +(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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: hashpartitioning(cs_order_number#50, cs_item_sk#49, 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#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_order_number#50 ASC NULLS FIRST, cs_item_sk#49 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#56, cr_order_number#57, cr_returned_date_sk#58] +(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#56, cr_order_number#57, cr_returned_date_sk#58] -Condition : (isnotnull(cr_order_number#57) AND isnotnull(cr_item_sk#56)) +(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)) + +(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] + +(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] -(52) CometProject -Input [3]: [cr_item_sk#56, cr_order_number#57, cr_returned_date_sk#58] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_item_sk#56, cr_order_number#57] +(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] -(53) CometExchange -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: hashpartitioning(cr_order_number#57, cr_item_sk#56, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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 -(54) CometSort -Input [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cr_item_sk#56, cr_order_number#57], [cr_order_number#57 ASC NULLS FIRST, cr_item_sk#56 ASC NULLS FIRST] +(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) -(55) CometSortMergeJoin -Left output [7]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [cr_item_sk#56, cr_order_number#57] -Arguments: [cs_order_number#50, cs_item_sk#49], [cr_order_number#57, cr_item_sk#56], LeftOuter +(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] -(56) CometFilter -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Condition : isnull(cr_order_number#57) +(56) ReusedExchange [Reuses operator id: 15] +Output [2]: [d_date_sk#53, d_year#54] -(57) CometProject -Input [9]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_order_number#50, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, cr_item_sk#56, cr_order_number#57] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] +(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 -(58) ReusedExchange [Reuses operator id: 15] -Output [2]: [d_date_sk#59, d_year#60] +(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] -(59) CometBroadcastHashJoin -Left output [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54] -Right output [2]: [d_date_sk#59, d_year#60] -Arguments: [cs_sold_date_sk#54], [d_date_sk#59], Inner, BuildRight +(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))] -(60) CometProject -Input [8]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, cs_sold_date_sk#54, d_date_sk#59, d_year#60] -Arguments: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60], [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] +(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] (61) CometHashAggregate -Input [6]: [cs_bill_customer_sk#48, cs_item_sk#49, cs_quantity#51, cs_wholesale_cost#52, cs_sales_price#53, d_year#60] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [partial_sum(cs_quantity#51), partial_sum(UnscaledValue(cs_wholesale_cost#52)), partial_sum(UnscaledValue(cs_sales_price#53))] - -(62) CometExchange -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Arguments: hashpartitioning(d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(63) CometColumnarToRow [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] - -(64) HashAggregate [codegen id : 4] -Input [6]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48, sum#61, sum#62, sum#63] -Keys [3]: [d_year#60, cs_item_sk#49, cs_bill_customer_sk#48] -Functions [3]: [sum(cs_quantity#51), sum(UnscaledValue(cs_wholesale_cost#52)), sum(UnscaledValue(cs_sales_price#53))] -Aggregate Attributes [3]: [sum(cs_quantity#51)#64, sum(UnscaledValue(cs_wholesale_cost#52))#65, sum(UnscaledValue(cs_sales_price#53))#66] -Results [6]: [d_year#60 AS cs_sold_year#67, cs_item_sk#49, cs_bill_customer_sk#48 AS cs_customer_sk#68, sum(cs_quantity#51)#64 AS cs_qty#69, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#52))#65,17,2) AS cs_wc#70, MakeDecimal(sum(UnscaledValue(cs_sales_price#53))#66,17,2) AS cs_sp#71] - -(65) Filter [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Condition : (coalesce(cs_qty#69, 0) > 0) - -(66) Sort [codegen id : 4] -Input [6]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] -Arguments: [cs_sold_year#67 ASC NULLS FIRST, cs_item_sk#49 ASC NULLS FIRST, cs_customer_sk#68 ASC NULLS FIRST], false, 0 - -(67) SortMergeJoin [codegen id : 5] -Left keys [3]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2] -Right keys [3]: [cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68] -Join type: Inner -Join condition: None - -(68) Project [codegen id : 5] -Output [13]: [round((cast(ss_qty#21 as double) / knownfloatingpointnormalized(normalizenanandzero(cast(coalesce((ws_qty#45 + cs_qty#69), 1) as double)))), 2) AS ratio#72, ss_qty#21 AS store_qty#73, ss_wc#22 AS store_wholesale_cost#74, ss_sp#23 AS store_sales_price#75, (coalesce(ws_qty#45, 0) + coalesce(cs_qty#69, 0)) AS other_chan_qty#76, (coalesce(ws_wc#46, 0.00) + coalesce(cs_wc#70, 0.00)) AS other_chan_wholesale_cost#77, (coalesce(ws_sp#47, 0.00) + coalesce(cs_sp#71, 0.00)) AS other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Input [15]: [ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23, ws_qty#45, ws_wc#46, ws_sp#47, cs_sold_year#67, cs_item_sk#49, cs_customer_sk#68, cs_qty#69, cs_wc#70, cs_sp#71] - -(69) TakeOrderedAndProject -Input [13]: [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78, ss_sold_year#20, ss_item_sk#1, ss_customer_sk#2, ss_qty#21, ss_wc#22, ss_sp#23] -Arguments: 100, [ss_sold_year#20 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#21 DESC NULLS LAST, ss_wc#22 DESC NULLS LAST, ss_sp#23 DESC NULLS LAST, other_chan_qty#76 ASC NULLS FIRST, other_chan_wholesale_cost#77 ASC NULLS FIRST, other_chan_sales_price#78 ASC NULLS FIRST, ratio#72 ASC NULLS FIRST], [ratio#72, store_qty#73, store_wholesale_cost#74, store_sales_price#75, other_chan_qty#76, other_chan_wholesale_cost#77, other_chan_sales_price#78] +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))] + +(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) + +(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] + +(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 + +(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] + +(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] + +(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 (73) -+- * CometColumnarToRow (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (71) ++- * CometColumnarToRow (70) + +- CometFilter (69) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (68) -(70) 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 -(71) 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)) -(72) CometColumnarToRow [codegen id : 1] +(70) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#12, d_year#13] -(73) 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 = 23 Hosting Expression = ws_sold_date_sk#30 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#54 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-spark4_0/q78/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt index 6516f29483..9d5ba53be1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/extended.txt @@ -1,63 +1,61 @@ TakeOrderedAndProject -+- Project - +- SortMergeJoin - :- Project - : +- SortMergeJoin - : :- Sort - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#1 AS ss_sold_year#2, ss_item_sk#3, ss_customer_sk#4, sum(ss_quantity#5)#6 AS ss_qty#7, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#8))#9,17,2) AS ss_wc#10, MakeDecimal(sum(UnscaledValue(ss_sales_price#11))#12,17,2) AS ss_sp#13)] - : : +- CometColumnarToRow - : : +- 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 - : +- Sort - : +- Filter - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#14 AS ws_sold_year#15, ws_item_sk#16, ws_bill_customer_sk#17 AS ws_customer_sk#18, sum(ws_quantity#19)#20 AS ws_qty#21, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#22))#23,17,2) AS ws_wc#24, MakeDecimal(sum(UnscaledValue(ws_sales_price#25))#26,17,2) AS ws_sp#27)] - : +- CometColumnarToRow - : +- 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 - +- Sort - +- Filter - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(d_year#28 AS cs_sold_year#29, cs_item_sk#30, cs_bill_customer_sk#31 AS cs_customer_sk#32, sum(cs_quantity#33)#34 AS cs_qty#35, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#36))#37,17,2) AS cs_wc#38, MakeDecimal(sum(UnscaledValue(cs_sales_price#39))#40,17,2) AS cs_sp#41)] - +- CometColumnarToRow ++- 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 @@ -79,4 +77,4 @@ TakeOrderedAndProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 59 out of 76 eligible operators (77%). Final plan contains 4 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-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt index 850bab3c02..273db28e77 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -1,90 +1,78 @@ 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 (5) + 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] - SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + CometColumnarToRow InputAdapter - WholeStageCodegen (3) - 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 (1) - 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 - 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 - 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] - InputAdapter - WholeStageCodegen (2) - 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 - 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 - InputAdapter - WholeStageCodegen (4) - 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 - 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] + 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 + 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 [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] + 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-spark4_0/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt index 1d8afd531a..224275b64b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (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) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* 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 @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(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))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(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#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +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 -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(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) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 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] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(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 -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(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) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(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] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(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] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 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] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], 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 -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(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] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(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 -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(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] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(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 -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#51, 16)) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(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] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(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] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(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] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(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] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(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)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(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] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(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)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(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#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +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 -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(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)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(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 -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(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)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(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] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(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] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], 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 -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(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] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(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 -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(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] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(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 -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#83, 16)) AS web_site_id#84] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(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 -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(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 -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(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 -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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 [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#118, id#119] -Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#126)#112, sum(returns#127)#113, sum(profit#128)#114] -Results [4]: [channel#118, sum(sales#126)#112 AS sales#129, sum(returns#127)#113 AS returns#130, sum(profit#128)#114 AS profit#131] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#118, sales#129, returns#130, profit#131] -Keys [1]: [channel#118] -Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(109) CometColumnarExchange -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#118, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [1]: [channel#118] -Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] -Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] -Results [5]: [channel#118, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -Keys [2]: [channel#151, id#152] -Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#159)#112, sum(returns#160)#113, sum(profit#161)#114] -Results [3]: [sum(sales#159)#112 AS sales#162, sum(returns#160)#113 AS returns#163, sum(profit#161)#114 AS profit#164] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#162, returns#163, profit#164] +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#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] + +(102) CometHashAggregate +Input [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [2]: [channel#97, id#98] +Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] + +(103) CometHashAggregate +Input [4]: [channel#97, sales#108, returns#109, profit#110] +Keys [1]: [channel#97] +Functions [3]: [partial_sum(sales#108), partial_sum(returns#109), partial_sum(profit#110)] + +(104) CometExchange +Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Arguments: hashpartitioning(channel#97, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Keys [1]: [channel#97] +Functions [3]: [sum(sales#108), sum(returns#109), sum(profit#110)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] + +(107) CometHashAggregate +Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Keys [2]: [channel#117, id#118] +Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] + +(108) CometHashAggregate +Input [3]: [sales#128, returns#129, profit#130] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] - -(116) CometColumnarExchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +(109) CometExchange +Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +(110) CometHashAggregate +Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#177, sum(returns#163)#178, sum(profit#164)#179] -Results [5]: [null AS channel#180, null AS id#181, sum(sales#162)#177 AS sales#182, sum(returns#163)#178 AS returns#183, sum(profit#164)#179 AS profit#184] +Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Child 1 Input [5]: [channel#97, id#140, sales#141, returns#142, profit#143] +Child 2 Input [5]: [channel#144, id#145, sales#146, returns#147, profit#148] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Arguments: hashpartitioning(channel#76, id#77, sales#137, returns#138, profit#139, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#137,returns#138,profit#139]), [channel#76, id#77, sales#137, returns#138, profit#139], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#137, returns#138, profit#139] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(126) CometFilter +(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)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 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-spark4_0/q80a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt index 605b9dac36..0d6844d93c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/extended.txt @@ -1,406 +1,392 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - : +- CometColumnarToRow - : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - : +- CometColumnarToRow - : +- CometExchange - : +- 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 : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate + : +- 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 - +- 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 + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 350 out of 386 eligible operators (90%). Final plan contains 18 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-spark4_0/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a.native_iceberg_compat/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - 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 (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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 + 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] - 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] - WholeStageCodegen (2) - 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 - 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 - WholeStageCodegen (3) - 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 - 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 - WholeStageCodegen (11) - 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] #19 - WholeStageCodegen (10) - 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 (17) - 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 #20 - WholeStageCodegen (16) - 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 + 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-spark4_0/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt index 1d8afd531a..224275b64b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (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) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* 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 @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co 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] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(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))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(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#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +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 -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(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) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(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) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 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] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(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 -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(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) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(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] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(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] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 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] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], 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 -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(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] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(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 -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(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] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(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 -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#51, 16)) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(cp_catalog_page_id#43, 16)) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(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] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(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] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(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] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(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 -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(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] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(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)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(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] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(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)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(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#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +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 -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(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)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(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] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 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] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(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 -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(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)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(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] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(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] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(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] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], 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 -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(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] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(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 -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(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] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(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 -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#83, 16)) AS web_site_id#84] +(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, static_invoke(CharVarcharCodegenUtils.readSidePadding(web_site_id#67, 16)) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(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 -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(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 -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(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 -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(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 [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#118, id#119, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] -Keys [2]: [channel#118, id#119] -Functions [3]: [sum(sales#126), sum(returns#127), sum(profit#128)] -Aggregate Attributes [3]: [sum(sales#126)#112, sum(returns#127)#113, sum(profit#128)#114] -Results [4]: [channel#118, sum(sales#126)#112 AS sales#129, sum(returns#127)#113 AS returns#130, sum(profit#128)#114 AS profit#131] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#118, sales#129, returns#130, profit#131] -Keys [1]: [channel#118] -Functions [3]: [partial_sum(sales#129), partial_sum(returns#130), partial_sum(profit#131)] -Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] -Results [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(109) CometColumnarExchange -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Arguments: hashpartitioning(channel#118, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#118, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] -Keys [1]: [channel#118] -Functions [3]: [sum(sales#129), sum(returns#130), sum(profit#131)] -Aggregate Attributes [3]: [sum(sales#129)#144, sum(returns#130)#145, sum(profit#131)#146] -Results [5]: [channel#118, null AS id#147, sum(sales#129)#144 AS sales#148, sum(returns#130)#145 AS returns#149, sum(profit#131)#146 AS profit#150] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#151, id#152, sum#153, isEmpty#154, sum#155, isEmpty#156, sum#157, isEmpty#158] -Keys [2]: [channel#151, id#152] -Functions [3]: [sum(sales#159), sum(returns#160), sum(profit#161)] -Aggregate Attributes [3]: [sum(sales#159)#112, sum(returns#160)#113, sum(profit#161)#114] -Results [3]: [sum(sales#159)#112 AS sales#162, sum(returns#160)#113 AS returns#163, sum(profit#161)#114 AS profit#164] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#162, returns#163, profit#164] +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#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] + +(102) CometHashAggregate +Input [8]: [channel#97, id#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Keys [2]: [channel#97, id#98] +Functions [3]: [sum(sales#105), sum(returns#106), sum(profit#107)] + +(103) CometHashAggregate +Input [4]: [channel#97, sales#108, returns#109, profit#110] +Keys [1]: [channel#97] +Functions [3]: [partial_sum(sales#108), partial_sum(returns#109), partial_sum(profit#110)] + +(104) CometExchange +Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Arguments: hashpartitioning(channel#97, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#97, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Keys [1]: [channel#97] +Functions [3]: [sum(sales#108), sum(returns#109), sum(profit#110)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] + +(107) CometHashAggregate +Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Keys [2]: [channel#117, id#118] +Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] + +(108) CometHashAggregate +Input [3]: [sales#128, returns#129, profit#130] Keys: [] -Functions [3]: [partial_sum(sales#162), partial_sum(returns#163), partial_sum(profit#164)] -Aggregate Attributes [6]: [sum#165, isEmpty#166, sum#167, isEmpty#168, sum#169, isEmpty#170] -Results [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] - -(116) CometColumnarExchange -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +(109) CometExchange +Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#171, isEmpty#172, sum#173, isEmpty#174, sum#175, isEmpty#176] +(110) CometHashAggregate +Input [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] Keys: [] -Functions [3]: [sum(sales#162), sum(returns#163), sum(profit#164)] -Aggregate Attributes [3]: [sum(sales#162)#177, sum(returns#163)#178, sum(profit#164)#179] -Results [5]: [null AS channel#180, null AS id#181, sum(sales#162)#177 AS sales#182, sum(returns#163)#178 AS returns#183, sum(profit#164)#179 AS profit#184] +Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Child 1 Input [5]: [channel#97, id#140, sales#141, returns#142, profit#143] +Child 2 Input [5]: [channel#144, id#145, sales#146, returns#147, profit#148] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Arguments: hashpartitioning(channel#76, id#77, sales#137, returns#138, profit#139, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Keys [5]: [channel#76, id#77, sales#137, returns#138, profit#139] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#137,returns#138,profit#139]), [channel#76, id#77, sales#137, returns#138, profit#139], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#137, returns#138, profit#139] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#137, returns#138, profit#139] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(126) CometFilter +(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)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 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-spark4_0/q80a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt index 605b9dac36..0d6844d93c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/extended.txt @@ -1,406 +1,392 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Union - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - : +- CometColumnarToRow - : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : : +- CometColumnarToRow - : : +- 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 - : :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : : +- CometColumnarToRow - : : +- 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 - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - : +- CometColumnarToRow - : +- CometExchange - : +- 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 : +- 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 [COMET: makedecimal is not supported, Unsupported result expressions found in: List(store channel AS channel#1, concat(store, s_store_id#2) AS id#3, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#5,17,2) AS sales#6, sum(coalesce(cast(sr_return_amt#7 as decimal(12,2)), 0.00))#8 AS returns#9, sum((ss_net_profit#10 - coalesce(cast(sr_net_loss#11 as decimal(12,2)), 0.00)))#12 AS profit#13)] - : +- CometColumnarToRow - : +- 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 - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(catalog channel AS channel#14, concat(catalog_page, cp_catalog_page_id#15) AS id#16, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#17))#18,17,2) AS sales#19, sum(coalesce(cast(cr_return_amount#20 as decimal(12,2)), 0.00))#21 AS returns#22, sum((cs_net_profit#23 - coalesce(cast(cr_net_loss#24 as decimal(12,2)), 0.00)))#25 AS profit#26)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(web channel AS channel#27, concat(web_site, web_site_id#28) AS id#29, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#30))#31,17,2) AS sales#32, sum(coalesce(cast(wr_return_amt#33 as decimal(12,2)), 0.00))#34 AS returns#35, sum((ws_net_profit#36 - coalesce(cast(wr_net_loss#37 as decimal(12,2)), 0.00)))#38 AS profit#39)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate + : +- 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 - +- 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 + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion -Comet accelerated 350 out of 386 eligible operators (90%). Final plan contains 18 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-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - 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 (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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 + 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] - 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] - WholeStageCodegen (2) - 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 - 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 - WholeStageCodegen (3) - 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 - 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 - WholeStageCodegen (11) - 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] #19 - WholeStageCodegen (10) - 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 (17) - 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 #20 - WholeStageCodegen (16) - 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 + 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-spark4_0/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt index 07622631b8..7f23c0ebbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt @@ -1,46 +1,41 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * CometColumnarToRow (39) - +- CometSort (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometColumnarExchange (35) - +- * HashAggregate (34) - +- Union (33) - :- * HashAggregate (18) - : +- * CometColumnarToRow (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) - :- * HashAggregate (25) - : +- * CometColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- * HashAggregate (22) - : +- * HashAggregate (21) - : +- * CometColumnarToRow (20) - : +- ReusedExchange (19) - +- * HashAggregate (32) - +- * CometColumnarToRow (31) - +- CometColumnarExchange (30) - +- * HashAggregate (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- ReusedExchange (26) +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 @@ -120,154 +115,126 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] 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] -(17) CometColumnarToRow [codegen id : 1] -Input [3]: [i_category#11, i_class#10, sum#12] - -(18) HashAggregate [codegen id : 1] +(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#11 AS i_category#15, i_class#10 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] -(19) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#20, i_class#21, sum#22] - -(20) CometColumnarToRow [codegen id : 2] -Input [3]: [i_category#20, i_class#21, sum#22] +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#13, i_class#14, sum#15] -(21) HashAggregate [codegen id : 2] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#13,17,2) AS total_sum#24, i_category#20] +(19) CometHashAggregate +Input [3]: [i_category#13, i_class#14, sum#15] +Keys [2]: [i_category#13, i_class#14] +Functions [1]: [sum(UnscaledValue(ws_net_paid#16))] -(22) HashAggregate [codegen id : 2] -Input [2]: [total_sum#24, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#24)] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [i_category#20, sum#27, isEmpty#28] +(20) CometHashAggregate +Input [2]: [total_sum#17, i_category#13] +Keys [1]: [i_category#13] +Functions [1]: [partial_sum(total_sum#17)] -(23) CometColumnarExchange -Input [3]: [i_category#20, sum#27, isEmpty#28] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometExchange +Input [3]: [i_category#13, sum#18, isEmpty#19] +Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] -Input [3]: [i_category#20, sum#27, isEmpty#28] +(22) CometHashAggregate +Input [3]: [i_category#13, sum#18, isEmpty#19] +Keys [1]: [i_category#13] +Functions [1]: [sum(total_sum#17)] -(25) HashAggregate [codegen id : 3] -Input [3]: [i_category#20, sum#27, isEmpty#28] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#24)] -Aggregate Attributes [1]: [sum(total_sum#24)#29] -Results [6]: [sum(total_sum#24)#29 AS total_sum#30, i_category#20, null AS i_class#31, 0 AS g_category#32, 1 AS g_class#33, 1 AS lochierarchy#34] - -(26) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#35, i_class#36, sum#37] - -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#35, i_class#36, sum#37] +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#20, i_class#21, sum#22] -(28) HashAggregate [codegen id : 4] -Input [3]: [i_category#35, i_class#36, sum#37] -Keys [2]: [i_category#35, i_class#36] -Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#13,17,2) AS total_sum#39] +(24) CometHashAggregate +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -(29) HashAggregate [codegen id : 4] -Input [1]: [total_sum#39] +(25) CometHashAggregate +Input [1]: [total_sum#24] Keys: [] -Functions [1]: [partial_sum(total_sum#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [2]: [sum#42, isEmpty#43] - -(30) CometColumnarExchange -Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Functions [1]: [partial_sum(total_sum#24)] -(31) CometColumnarToRow [codegen id : 5] -Input [2]: [sum#42, isEmpty#43] +(26) CometExchange +Input [2]: [sum#25, isEmpty#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 5] -Input [2]: [sum#42, isEmpty#43] +(27) CometHashAggregate +Input [2]: [sum#25, isEmpty#26] Keys: [] -Functions [1]: [sum(total_sum#39)] -Aggregate Attributes [1]: [sum(total_sum#39)#44] -Results [6]: [sum(total_sum#39)#44 AS total_sum#45, null AS i_category#46, null AS i_class#47, 1 AS g_category#48, 1 AS g_class#49, 2 AS lochierarchy#50] +Functions [1]: [sum(total_sum#24)] -(33) Union +(28) CometUnion +Child 0 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Child 1 Input [6]: [total_sum#33, i_category#13, i_class#34, g_category#35, g_class#36, lochierarchy#37] +Child 2 Input [6]: [total_sum#38, i_category#39, i_class#40, g_category#41, g_class#42, lochierarchy#43] -(34) HashAggregate [codegen id : 6] -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#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -(35) 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=4] +(30) CometExchange +Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Arguments: hashpartitioning(total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(36) 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] +(31) CometHashAggregate +Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] Functions: [] -(37) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(32) CometExchange +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(38) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] +(33) CometSort +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#27 DESC NULLS LAST] -(39) CometColumnarToRow [codegen id : 7] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +(34) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -(40) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] +(35) Window +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: [rank(total_sum#27) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#27 DESC NULLS LAST] -(41) Project [codegen id : 8] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] +(36) Project [codegen id : 2] +Output [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] +Input [6]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44, rank_within_parent#45] -(42) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(37) TakeOrderedAndProject +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] +Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN i_category#28 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * CometColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(44) CometFilter +(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) CometProject +(40) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(47) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#5] 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-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt index 2044af2e4e..46c47555a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt @@ -5,88 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) as decimal(27,2)) AS total_sum#3, i_category#4 AS i_category#5, i_class#6 AS i_class#7, 0 AS g_category#8, 0 AS g_class#9, 0 AS lochierarchy#10)] - : +- CometColumnarToRow - : +- CometExchange + +- 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 - : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#11))#2,17,2) AS total_sum#12, i_category#13)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#14))#2,17,2) AS total_sum#15)] - +- CometColumnarToRow - +- 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 + : +- 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 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt index d9db895fba..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt @@ -1,66 +1,49 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (7) + WholeStageCodegen (1) 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 (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - 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 (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - 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 + 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-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index 07622631b8..7f23c0ebbf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -1,46 +1,41 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * CometColumnarToRow (39) - +- CometSort (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometColumnarExchange (35) - +- * HashAggregate (34) - +- Union (33) - :- * HashAggregate (18) - : +- * CometColumnarToRow (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) - :- * HashAggregate (25) - : +- * CometColumnarToRow (24) - : +- CometColumnarExchange (23) - : +- * HashAggregate (22) - : +- * HashAggregate (21) - : +- * CometColumnarToRow (20) - : +- ReusedExchange (19) - +- * HashAggregate (32) - +- * CometColumnarToRow (31) - +- CometColumnarExchange (30) - +- * HashAggregate (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- ReusedExchange (26) +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 @@ -120,154 +115,126 @@ Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] 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] -(17) CometColumnarToRow [codegen id : 1] -Input [3]: [i_category#11, i_class#10, sum#12] - -(18) HashAggregate [codegen id : 1] +(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#11 AS i_category#15, i_class#10 AS i_class#16, 0 AS g_category#17, 0 AS g_class#18, 0 AS lochierarchy#19] -(19) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#20, i_class#21, sum#22] - -(20) CometColumnarToRow [codegen id : 2] -Input [3]: [i_category#20, i_class#21, sum#22] +(18) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#13, i_class#14, sum#15] -(21) HashAggregate [codegen id : 2] -Input [3]: [i_category#20, i_class#21, sum#22] -Keys [2]: [i_category#20, i_class#21] -Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#23))#13] -Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#23))#13,17,2) AS total_sum#24, i_category#20] +(19) CometHashAggregate +Input [3]: [i_category#13, i_class#14, sum#15] +Keys [2]: [i_category#13, i_class#14] +Functions [1]: [sum(UnscaledValue(ws_net_paid#16))] -(22) HashAggregate [codegen id : 2] -Input [2]: [total_sum#24, i_category#20] -Keys [1]: [i_category#20] -Functions [1]: [partial_sum(total_sum#24)] -Aggregate Attributes [2]: [sum#25, isEmpty#26] -Results [3]: [i_category#20, sum#27, isEmpty#28] +(20) CometHashAggregate +Input [2]: [total_sum#17, i_category#13] +Keys [1]: [i_category#13] +Functions [1]: [partial_sum(total_sum#17)] -(23) CometColumnarExchange -Input [3]: [i_category#20, sum#27, isEmpty#28] -Arguments: hashpartitioning(i_category#20, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +(21) CometExchange +Input [3]: [i_category#13, sum#18, isEmpty#19] +Arguments: hashpartitioning(i_category#13, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(24) CometColumnarToRow [codegen id : 3] -Input [3]: [i_category#20, sum#27, isEmpty#28] +(22) CometHashAggregate +Input [3]: [i_category#13, sum#18, isEmpty#19] +Keys [1]: [i_category#13] +Functions [1]: [sum(total_sum#17)] -(25) HashAggregate [codegen id : 3] -Input [3]: [i_category#20, sum#27, isEmpty#28] -Keys [1]: [i_category#20] -Functions [1]: [sum(total_sum#24)] -Aggregate Attributes [1]: [sum(total_sum#24)#29] -Results [6]: [sum(total_sum#24)#29 AS total_sum#30, i_category#20, null AS i_class#31, 0 AS g_category#32, 1 AS g_class#33, 1 AS lochierarchy#34] - -(26) ReusedExchange [Reuses operator id: 16] -Output [3]: [i_category#35, i_class#36, sum#37] - -(27) CometColumnarToRow [codegen id : 4] -Input [3]: [i_category#35, i_class#36, sum#37] +(23) ReusedExchange [Reuses operator id: 16] +Output [3]: [i_category#20, i_class#21, sum#22] -(28) HashAggregate [codegen id : 4] -Input [3]: [i_category#35, i_class#36, sum#37] -Keys [2]: [i_category#35, i_class#36] -Functions [1]: [sum(UnscaledValue(ws_net_paid#38))] -Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#38))#13] -Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#38))#13,17,2) AS total_sum#39] +(24) CometHashAggregate +Input [3]: [i_category#20, i_class#21, sum#22] +Keys [2]: [i_category#20, i_class#21] +Functions [1]: [sum(UnscaledValue(ws_net_paid#23))] -(29) HashAggregate [codegen id : 4] -Input [1]: [total_sum#39] +(25) CometHashAggregate +Input [1]: [total_sum#24] Keys: [] -Functions [1]: [partial_sum(total_sum#39)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [2]: [sum#42, isEmpty#43] - -(30) CometColumnarExchange -Input [2]: [sum#42, isEmpty#43] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Functions [1]: [partial_sum(total_sum#24)] -(31) CometColumnarToRow [codegen id : 5] -Input [2]: [sum#42, isEmpty#43] +(26) CometExchange +Input [2]: [sum#25, isEmpty#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) HashAggregate [codegen id : 5] -Input [2]: [sum#42, isEmpty#43] +(27) CometHashAggregate +Input [2]: [sum#25, isEmpty#26] Keys: [] -Functions [1]: [sum(total_sum#39)] -Aggregate Attributes [1]: [sum(total_sum#39)#44] -Results [6]: [sum(total_sum#39)#44 AS total_sum#45, null AS i_category#46, null AS i_class#47, 1 AS g_category#48, 1 AS g_class#49, 2 AS lochierarchy#50] +Functions [1]: [sum(total_sum#24)] -(33) Union +(28) CometUnion +Child 0 Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Child 1 Input [6]: [total_sum#33, i_category#13, i_class#34, g_category#35, g_class#36, lochierarchy#37] +Child 2 Input [6]: [total_sum#38, i_category#39, i_class#40, g_category#41, g_class#42, lochierarchy#43] -(34) HashAggregate [codegen id : 6] -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#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#14, i_category#15, i_class#16, g_category#17, g_class#18, lochierarchy#19] -(35) 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=4] +(30) CometExchange +Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Arguments: hashpartitioning(total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(36) 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] +(31) CometHashAggregate +Input [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] +Keys [6]: [total_sum#27, i_category#28, i_class#29, g_category#30, g_class#31, lochierarchy#32] Functions: [] -(37) CometExchange -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(32) CometExchange +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: hashpartitioning(lochierarchy#32, _w0#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(38) CometSort -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] +(33) CometSort +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44], [lochierarchy#32 ASC NULLS FIRST, _w0#44 ASC NULLS FIRST, total_sum#27 DESC NULLS LAST] -(39) CometColumnarToRow [codegen id : 7] -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +(34) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] -(40) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] +(35) Window +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44] +Arguments: [rank(total_sum#27) windowspecdefinition(lochierarchy#32, _w0#44, total_sum#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#45], [lochierarchy#32, _w0#44], [total_sum#27 DESC NULLS LAST] -(41) Project [codegen id : 8] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] +(36) Project [codegen id : 2] +Output [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] +Input [6]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, _w0#44, rank_within_parent#45] -(42) TakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(37) TakeOrderedAndProject +Input [5]: [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] +Arguments: 100, [lochierarchy#32 DESC NULLS LAST, CASE WHEN (lochierarchy#32 = 0) THEN i_category#28 END ASC NULLS FIRST, rank_within_parent#45 ASC NULLS FIRST], [total_sum#27, i_category#28, i_class#29, lochierarchy#32, rank_within_parent#45] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (47) -+- * CometColumnarToRow (46) - +- CometProject (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (42) ++- * CometColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (38) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(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 -(44) CometFilter +(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) CometProject +(40) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(46) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(47) BroadcastExchange +(42) BroadcastExchange Input [1]: [d_date_sk#5] 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-spark4_0/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt index 2044af2e4e..46c47555a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt @@ -5,88 +5,83 @@ TakeOrderedAndProject +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) as decimal(27,2)) AS total_sum#3, i_category#4 AS i_category#5, i_class#6 AS i_class#7, 0 AS g_category#8, 0 AS g_class#9, 0 AS lochierarchy#10)] - : +- CometColumnarToRow - : +- CometExchange + +- 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 - : +- 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 - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#11))#2,17,2) AS total_sum#12, i_category#13)] - : +- CometColumnarToRow - : +- 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 - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#14))#2,17,2) AS total_sum#15)] - +- CometColumnarToRow - +- 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 + : +- 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 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index d9db895fba..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -1,66 +1,49 @@ TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (8) + WholeStageCodegen (2) Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] InputAdapter Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (7) + WholeStageCodegen (1) 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 (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - 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 - 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] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - 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 (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - 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 + 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-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt index 4b24124687..7aac2f05bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (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.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 @@ -104,75 +103,70 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) 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=3] +(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] -(25) 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] +(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] -(26) CometColumnarToRow [codegen id : 4] -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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) 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 -(28) CometFilter +(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)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt index f00a5af531..3dbaf2e346 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt @@ -5,30 +5,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt index 1a79190465..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,42 +1,39 @@ -WholeStageCodegen (4) +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 (3) + 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 (2) + 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 (1) - 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 - 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] + 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-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index 4b24124687..7aac2f05bf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (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) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (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.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 @@ -104,75 +103,70 @@ Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] 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) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, sum#16] - -(18) HashAggregate [codegen id : 1] +(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] -(19) 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=2] +(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] -(20) 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] +(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] -(21) CometColumnarToRow [codegen id : 2] -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] -(22) 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] +(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] -(23) Project [codegen id : 3] -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] -(24) 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=3] +(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] -(25) 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] +(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] -(26) CometColumnarToRow [codegen id : 4] -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 ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) 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 -(28) CometFilter +(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)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] 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-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt index f00a5af531..3dbaf2e346 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt @@ -5,30 +5,29 @@ CometColumnarToRow +- Window [COMET: WindowExec is not fully compatible with Spark (Native WindowExec has known correctness issues). To enable it anyway, set spark.comet.operator.WindowExec.allowIncompatible=true. For more information, refer to the Comet Compatibility Guide (https://datafusion.apache.org/comet/user-guide/compatibility.html).] +- CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- 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 + +- 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 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 1a79190465..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -1,42 +1,39 @@ -WholeStageCodegen (4) +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 (3) + 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 (2) + 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 (1) - 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 - 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] + 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]